-
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
Merged
Merged
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
f6ad68f
Do not create new Executor everytime createRunner
Abacn 443e38e
reset executorService after shutdown
Abacn d55a68e
Switch to use newScheduledThreadPool; guard ses with AtomicReference
Abacn 3b08a1f
Partially revert changes on flink and samza runner
Abacn 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 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
This file contains 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
This file contains 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 |
---|---|---|
|
@@ -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); | ||
|
||
|
@@ -172,7 +182,7 @@ public <AdditionalOutputT> void outputWindowedValue( | |
} | ||
}, | ||
NullSideInputReader.empty(), | ||
Executors.newSingleThreadScheduledExecutor(Executors.defaultThreadFactory()), | ||
ses, | ||
10000, | ||
Duration.standardSeconds(10), | ||
() -> { | ||
|
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.
remembered a cleaner way to do this:
private final Supplier sesSupplier =
Suppliers.memoize(
() -> Executors.newScheduledThreadPool(
Runtime.getRuntime().availableProcessors(),
new ThreadFactoryBuilder().setNameFormat("df-sdf-executor-%d").build())));
and then just ses.get() below
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.
thanks, good to learn, for now merging it before release cut