-
Notifications
You must be signed in to change notification settings - Fork 4.3k
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
Do not create new Executor everytime createRunner #32272
Changes from 2 commits
f6ad68f
443e38e
d55a68e
3b08a1f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -28,6 +28,7 @@ | |
import java.util.List; | ||
import java.util.Map; | ||
import java.util.concurrent.Executors; | ||
import java.util.concurrent.ScheduledExecutorService; | ||
import org.apache.beam.runners.core.DoFnRunner; | ||
import org.apache.beam.runners.core.DoFnRunners.OutputManager; | ||
import org.apache.beam.runners.core.KeyedWorkItem; | ||
|
@@ -56,6 +57,8 @@ | |
import org.apache.beam.sdk.values.PCollectionView; | ||
import org.apache.beam.sdk.values.TupleTag; | ||
import org.apache.beam.sdk.values.WindowingStrategy; | ||
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; | ||
import org.checkerframework.checker.nullness.qual.MonotonicNonNull; | ||
import org.joda.time.Duration; | ||
import org.joda.time.Instant; | ||
|
||
|
@@ -115,6 +118,8 @@ private static class ProcessFnExtractor implements UserParDoFnFactory.DoFnExtrac | |
private static class SplittableDoFnRunnerFactory< | ||
InputT, OutputT, RestrictionT, PositionT, WatermarkEstimatorStateT> | ||
implements DoFnRunnerFactory<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>, OutputT> { | ||
private transient @MonotonicNonNull ScheduledExecutorService ses = null; | ||
|
||
@Override | ||
public DoFnRunner<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>, OutputT> createRunner( | ||
DoFn<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>, OutputT> fn, | ||
|
@@ -131,6 +136,11 @@ public DoFnRunner<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>, OutputT> crea | |
OutputManager outputManager, | ||
DoFnSchemaInformation doFnSchemaInformation, | ||
Map<String, PCollectionView<?>> sideInputMapping) { | ||
if (this.ses == null) { | ||
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. if this factory possibly called concurrently? might need some synchronization if so? 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. I am not sure but now guard with AtomicReference |
||
this.ses = | ||
Executors.newSingleThreadScheduledExecutor( | ||
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. I don't think we want a single threaded one here, because I believe the factory vends many different dofnrunner which will want some parallel threads for splits. 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. changed to use Executors.newScheduledThreadPool |
||
new ThreadFactoryBuilder().setNameFormat("df-sdf-executor-%d").build()); | ||
} | ||
ProcessFn<InputT, OutputT, RestrictionT, PositionT, WatermarkEstimatorStateT> processFn = | ||
(ProcessFn<InputT, OutputT, RestrictionT, PositionT, WatermarkEstimatorStateT>) fn; | ||
processFn.setStateInternalsFactory(key -> (StateInternals) stepContext.stateInternals()); | ||
|
@@ -162,7 +172,7 @@ public <T> void outputWindowedValue( | |
} | ||
}, | ||
sideInputReader, | ||
Executors.newSingleThreadScheduledExecutor(Executors.defaultThreadFactory()), | ||
ses, | ||
// Commit at least once every 10 seconds or 10k records. This keeps the watermark | ||
// advancing smoothly, and ensures that not too much work will have to be reprocessed | ||
// in the event of a crash. | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -20,6 +20,7 @@ | |
import java.util.Collection; | ||
import java.util.Collections; | ||
import java.util.concurrent.Executors; | ||
import java.util.concurrent.ScheduledExecutorService; | ||
import org.apache.beam.runners.core.DoFnRunner; | ||
import org.apache.beam.runners.core.DoFnRunners; | ||
import org.apache.beam.runners.core.KeyedWorkItem; | ||
|
@@ -49,9 +50,11 @@ | |
import org.apache.beam.sdk.values.PCollection.IsBounded; | ||
import org.apache.beam.sdk.values.TupleTag; | ||
import org.apache.beam.sdk.values.WindowingStrategy; | ||
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; | ||
import org.apache.samza.config.Config; | ||
import org.apache.samza.context.Context; | ||
import org.apache.samza.operators.Scheduler; | ||
import org.checkerframework.checker.nullness.qual.MonotonicNonNull; | ||
import org.joda.time.Duration; | ||
import org.joda.time.Instant; | ||
import org.slf4j.Logger; | ||
|
@@ -81,6 +84,7 @@ public class SplittableParDoProcessKeyedElementsOp< | |
private transient SamzaTimerInternalsFactory<byte[]> timerInternalsFactory; | ||
private transient DoFnRunner<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>, OutputT> fnRunner; | ||
private transient SamzaPipelineOptions pipelineOptions; | ||
private transient @MonotonicNonNull ScheduledExecutorService ses = null; | ||
|
||
public SplittableParDoProcessKeyedElementsOp( | ||
TupleTag<OutputT> mainOutputTag, | ||
|
@@ -137,6 +141,12 @@ public void open( | |
isBounded, | ||
pipelineOptions); | ||
|
||
if (this.ses == null) { | ||
this.ses = | ||
Executors.newSingleThreadScheduledExecutor( | ||
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. not yet using this below 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. fixed the mistake |
||
new ThreadFactoryBuilder().setNameFormat("samza-sdf-executor-%d").build()); | ||
} | ||
|
||
final KeyedInternals<byte[]> keyedInternals = | ||
new KeyedInternals<>(stateInternalsFactory, timerInternalsFactory); | ||
|
||
|
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.
added debug log in both branch and run a sample job, saw
indicating this change is effective