-
Notifications
You must be signed in to change notification settings - Fork 526
feat(sparkey): Deduplicate SparkeyReader across DoFn clones #5918
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
+37
−9
Closed
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change | ||||||||
|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -32,6 +32,7 @@ import org.apache.beam.sdk.util.CoderUtils | |||||||||
| import org.apache.beam.sdk.values.PCollectionView | ||||||||||
| import org.slf4j.LoggerFactory | ||||||||||
|
|
||||||||||
| import java.util.concurrent.{CompletableFuture, ConcurrentHashMap} | ||||||||||
| import scala.util.hashing.MurmurHash3 | ||||||||||
|
|
||||||||||
| /** | ||||||||||
|
|
@@ -545,12 +546,11 @@ package object sparkey extends SparkeyReaderInstances with SparkeyCoders { | |||||||||
| mapFn: SparkeyReader => T | ||||||||||
| ) extends SideInput[T] { | ||||||||||
| override def updateCacheOnGlobalWindow: Boolean = false | ||||||||||
| override def get[I, O](context: DoFn[I, O]#ProcessContext): T = | ||||||||||
| mapFn( | ||||||||||
| SparkeySideInput.checkMemory( | ||||||||||
| context.sideInput(view).getReader(RemoteFileUtil.create(context.getPipelineOptions)) | ||||||||||
| ) | ||||||||||
| ) | ||||||||||
| override def get[I, O](context: DoFn[I, O]#ProcessContext): T = { | ||||||||||
| val uri = context.sideInput(view) | ||||||||||
| val rfu = RemoteFileUtil.create(context.getPipelineOptions) | ||||||||||
| mapFn(SparkeySideInput.getOrCreateReader(uri, rfu)) | ||||||||||
| } | ||||||||||
| } | ||||||||||
|
|
||||||||||
| /** | ||||||||||
|
|
@@ -561,8 +561,10 @@ package object sparkey extends SparkeyReaderInstances with SparkeyCoders { | |||||||||
| extends SideInput[SparkeyMap[K, V]] { | ||||||||||
| override def updateCacheOnGlobalWindow: Boolean = false | ||||||||||
| override def get[I, O](context: DoFn[I, O]#ProcessContext): SparkeyMap[K, V] = { | ||||||||||
| val uri = context.sideInput(view) | ||||||||||
| val rfu = RemoteFileUtil.create(context.getPipelineOptions) | ||||||||||
| new SparkeyMap( | ||||||||||
| context.sideInput(view).getReader(RemoteFileUtil.create(context.getPipelineOptions)), | ||||||||||
| SparkeySideInput.getOrCreateReader(uri, rfu), | ||||||||||
| CoderMaterializer.beam(context.getPipelineOptions, Coder[K]), | ||||||||||
| CoderMaterializer.beam(context.getPipelineOptions, Coder[V]) | ||||||||||
| ) | ||||||||||
|
|
@@ -576,15 +578,41 @@ package object sparkey extends SparkeyReaderInstances with SparkeyCoders { | |||||||||
| private class LargeSetSideInput[K: Coder](val view: PCollectionView[SparkeyUri]) | ||||||||||
| extends SideInput[SparkeySet[K]] { | ||||||||||
| override def updateCacheOnGlobalWindow: Boolean = false | ||||||||||
| override def get[I, O](context: DoFn[I, O]#ProcessContext): SparkeySet[K] = | ||||||||||
| override def get[I, O](context: DoFn[I, O]#ProcessContext): SparkeySet[K] = { | ||||||||||
| val uri = context.sideInput(view) | ||||||||||
| val rfu = RemoteFileUtil.create(context.getPipelineOptions) | ||||||||||
| new SparkeySet( | ||||||||||
| context.sideInput(view).getReader(RemoteFileUtil.create(context.getPipelineOptions)), | ||||||||||
| SparkeySideInput.getOrCreateReader(uri, rfu), | ||||||||||
| CoderMaterializer.beam(context.getPipelineOptions, Coder[K]) | ||||||||||
| ) | ||||||||||
| } | ||||||||||
| } | ||||||||||
|
|
||||||||||
| // Readers are cached for the lifetime of the JVM and never closed. This is intentional: | ||||||||||
| // Beam side inputs have no close/teardown lifecycle, and in batch pipelines the JVM exits | ||||||||||
| // when the pipeline finishes. | ||||||||||
| // Note: the cache is keyed by URI path only. If the same path is rewritten with different | ||||||||||
| // data and a new pipeline is run in the same JVM (e.g. DirectRunner, REPL), stale readers | ||||||||||
| // will be returned. This is acceptable for Dataflow batch (one pipeline per JVM). | ||||||||||
| private object SparkeySideInput { | ||||||||||
| private val logger = LoggerFactory.getLogger(this.getClass) | ||||||||||
|
|
||||||||||
| private val readerCache = | ||||||||||
| new ConcurrentHashMap[String, CompletableFuture[SparkeyReader]]() | ||||||||||
|
|
||||||||||
| def getOrCreateReader(uri: SparkeyUri, rfu: RemoteFileUtil): SparkeyReader = | ||||||||||
| readerCache | ||||||||||
| .computeIfAbsent( | ||||||||||
| uri.path, | ||||||||||
| _ => | ||||||||||
| CompletableFuture.supplyAsync { () => | ||||||||||
| val reader = uri.getReader(rfu) | ||||||||||
| checkMemory(reader) | ||||||||||
| reader | ||||||||||
|
Comment on lines
+609
to
+611
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
Can this be simplified? |
||||||||||
| } | ||||||||||
| ) | ||||||||||
| .join() | ||||||||||
|
|
||||||||||
| def checkMemory(reader: SparkeyReader): SparkeyReader = { | ||||||||||
| val memoryBytes = java.lang.management.ManagementFactory.getOperatingSystemMXBean | ||||||||||
| .asInstanceOf[com.sun.management.OperatingSystemMXBean] | ||||||||||
|
|
||||||||||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Note: using a future here to support loading multiple readers in parallel if they are expensive to create - currently that's not true, but if we add loading/locking/load to heap later it might be.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
🤔 But is the CompletableFuture any relevant if its call is always followed by
.join()? I wonder removing CompletableFuture might be simpler.. If keeping, then consider leaving a comment for its motivation.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hm, perhaps you're right and I reasoned incorrectly about this - my hope was that multiple threads loading the same sideinputs would still allow for loading multiple things in parallel, but if the file ordering is stable, that doesn't actually help. Will think more about this, and potentially simplify