Skip to content
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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

/**
Expand Down Expand Up @@ -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))
}
}

/**
Expand All @@ -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])
)
Expand All @@ -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 { () =>
Copy link
Copy Markdown
Member Author

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.

Copy link
Copy Markdown
Contributor

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.

Copy link
Copy Markdown
Member Author

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

val reader = uri.getReader(rfu)
checkMemory(reader)
reader
Comment on lines +609 to +611
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
val reader = uri.getReader(rfu)
checkMemory(reader)
reader
checkMemory(uri.getReader(rfu))

Can this be simplified?

}
)
.join()

def checkMemory(reader: SparkeyReader): SparkeyReader = {
val memoryBytes = java.lang.management.ManagementFactory.getOperatingSystemMXBean
.asInstanceOf[com.sun.management.OperatingSystemMXBean]
Expand Down
Loading