Skip to content
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

#26395 Disabling possibly unnecessary prefetching during GroupIntoBatches by using an experimental flag #26618

Closed
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 @@ -26,6 +26,8 @@
import javax.annotation.Nullable;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.options.ExperimentalOptions;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.state.BagState;
import org.apache.beam.sdk.state.CombiningState;
import org.apache.beam.sdk.state.StateSpec;
Expand Down Expand Up @@ -110,6 +112,10 @@
public class GroupIntoBatches<K, InputT>
extends PTransform<PCollection<KV<K, InputT>>, PCollection<KV<K, Iterable<InputT>>>> {

/** Experiment to "avoid possibly unnecessary prefetching". */
public static final String AVOID_POSSIBLY_UNNECESSARY_PREFETCHING =
"avoid_possibly_unnecessary_prefetching";
Copy link
Contributor

Choose a reason for hiding this comment

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

consider a descriptive naming: disable_groupintobatches_preferch


/**
* Wrapper class for batching parameters supplied by users. Shared by both {@link
* GroupIntoBatches} and {@link GroupIntoBatches.WithShardedKey}.
Expand Down Expand Up @@ -343,7 +349,8 @@ public PCollection<KV<K, Iterable<InputT>>> expand(PCollection<KV<K, InputT>> in
weigher,
params.getMaxBufferingDuration(),
allowedLateness,
valueCoder)));
valueCoder,
input.getPipeline().getOptions())));
}

@VisibleForTesting
Expand Down Expand Up @@ -418,7 +425,8 @@ private static class GroupIntoBatchesDoFn<K, InputT>
@Nullable SerializableFunction<InputT, Long> weigher,
Duration maxBufferingDuration,
Duration allowedLateness,
Coder<InputT> inputValueCoder) {
Coder<InputT> inputValueCoder,
PipelineOptions options) {
this.batchSize = batchSize;
this.batchSizeBytes = batchSizeBytes;
this.weigher = weigher;
Expand Down Expand Up @@ -457,8 +465,14 @@ public long apply(long left, long right) {
this.timerTsSpec = StateSpecs.value();
this.minBufferedTsSpec = StateSpecs.combining(minCombineFn);

// Prefetch every 20% of batchSize elements. Do not prefetch if batchSize is too little
this.prefetchFrequency = ((batchSize / 5) <= 1) ? Long.MAX_VALUE : (batchSize / 5);
// Prefetch every 20% of batchSize elements. Do not prefetch ...
if (ExperimentalOptions.hasExperiment(options, AVOID_POSSIBLY_UNNECESSARY_PREFETCHING)) {
// ... if we have disabled prefetching
this.prefetchFrequency = Long.MAX_VALUE;
} else {
// ... or if the batchSize is too little
this.prefetchFrequency = ((batchSize / 5) <= 1) ? Long.MAX_VALUE : (batchSize / 5);
}
}

@Override
Expand Down