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

Optimize Datastream for batch #31950

Closed
wants to merge 38 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
38 commits
Select commit Hold shift + click to select a range
44d1271
Limit max split size for bounded sources in Datastream API
jto Jul 23, 2024
82e4a78
[Flink] Set return type of bounded sources
jto Jul 23, 2024
9641b2a
[Flink] Use a lazy split enumerator for bounded sources
jto Jul 23, 2024
4c71518
spotless
jto Jul 26, 2024
362b649
Revert "Limit max split size for bounded sources in Datastream API"
jto Jul 26, 2024
57736fe
[Flink] make ToKeyedWorkItem part of the DoFnOperator
jto Aug 6, 2024
d1dbd5a
[Flink] refactor CombinePerKeyTranslator
jto Aug 8, 2024
8560709
[Flink] Combine before Reduce (no side-input only)
jto Aug 9, 2024
90485c7
[WIP] combine before reduce
jto Aug 13, 2024
0786acf
Revert "[WIP] combine before reduce"
jto Aug 13, 2024
78da517
[WIP] combine before reduce (again)
jto Aug 13, 2024
5e7b170
[WIP] Also fire timer on finished bundle
jto Aug 13, 2024
7e82123
Revert "[WIP] Also fire timer on finished bundle"
jto Aug 14, 2024
6be49e0
Revert "[WIP] combine before reduce (again)"
jto Aug 14, 2024
7da29a8
[FLink] Set default maxParallelism to parallelism in batch to avoid skew
jto Aug 19, 2024
4faa6d0
[Flink] Implement partial reduce
jto Aug 19, 2024
4cbd239
[Flink] remove toBinaryKV
jto Aug 19, 2024
7e7b300
[Flink] fastercopy everywhere
jto Aug 20, 2024
0a634bf
[Flink] Avoid re-evaluating options every time a new state is stored
jto Aug 20, 2024
03a1996
Avoid re-serializing trigger on every element in GroupAlsoByWindowVia…
jto Aug 20, 2024
e885cb2
Cache stringKey in StateNamespaces.WindowNamespace
jto Aug 20, 2024
164973c
Revert "[Flink] fastercopy everywhere"
jto Aug 20, 2024
b37aa7d
Revert "Cache stringKey in StateNamespaces.WindowNamespace"
jto Aug 20, 2024
7e423e7
[Flink] dead code cleanup
jto Aug 20, 2024
7f0983d
[Flink] spotless
jto Aug 20, 2024
8751150
[Flink] Only serialize namespace keys if necessary
jto Aug 21, 2024
dec75f6
[Flink] pre-combine before GBK
jto Aug 23, 2024
b3b7c2c
[Flink] persistent PartialReduceBundleOperator operator state
jto Aug 23, 2024
d0e7cb3
[Flink] lower default max bundle size in batch mode
jto Aug 23, 2024
fb9ef12
[Flink] disable bundles in batch mode
jto Aug 26, 2024
7fc687f
[Flink] force slot sharing group for source in batch mode
jto Aug 27, 2024
0b07e4a
[Flink] fix lazy enumerator package
jto Aug 28, 2024
686a100
[Flink] implement combine for reduce with side input
jto Aug 28, 2024
772f4af
[Flink] further reduce default bundle size in batch
jto Aug 28, 2024
06dd507
[Flink] fix licence
jto Aug 28, 2024
3b1c6e9
[Flink] spotless
jto Aug 28, 2024
d7d3a7f
[Flink] checkstyle
jto Aug 28, 2024
2c7f737
[Flink] spotless
jto Aug 29, 2024
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 @@ -18,6 +18,7 @@
package org.apache.beam.runners.core;

import java.util.Collection;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine;
import org.apache.beam.runners.core.triggers.TriggerStateMachines;
import org.apache.beam.sdk.transforms.DoFn;
Expand All @@ -41,6 +42,7 @@ public class GroupAlsoByWindowViaWindowSetNewDoFn<
extends DoFn<RinT, KV<K, OutputT>> {

private static final long serialVersionUID = 1L;
private final RunnerApi.Trigger triggerProto;

public static <K, InputT, OutputT, W extends BoundedWindow>
DoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> create(
Expand Down Expand Up @@ -86,6 +88,7 @@ public GroupAlsoByWindowViaWindowSetNewDoFn(
this.windowingStrategy = noWildcard;
this.reduceFn = reduceFn;
this.stateInternalsFactory = stateInternalsFactory;
this.triggerProto = TriggerTranslation.toProto(windowingStrategy.getTrigger());
}

private OutputWindowedValue<KV<K, OutputT>> outputWindowedValue() {
Expand Down Expand Up @@ -123,9 +126,7 @@ public void processElement(ProcessContext c) throws Exception {
new ReduceFnRunner<>(
key,
windowingStrategy,
ExecutableTriggerStateMachine.create(
TriggerStateMachines.stateMachineForTrigger(
TriggerTranslation.toProto(windowingStrategy.getTrigger()))),
ExecutableTriggerStateMachine.create(TriggerStateMachines.stateMachineForTrigger(triggerProto)),
stateInternals,
timerInternals,
outputWindowedValue(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,23 +50,16 @@ public class CoderTypeSerializer<T> extends TypeSerializer<T> {

private final Coder<T> coder;

/**
* {@link SerializablePipelineOptions} deserialization will cause {@link
* org.apache.beam.sdk.io.FileSystems} registration needed for {@link
* org.apache.beam.sdk.transforms.Reshuffle} translation.
*/
private final SerializablePipelineOptions pipelineOptions;

private final boolean fasterCopy;

public CoderTypeSerializer(Coder<T> coder, SerializablePipelineOptions pipelineOptions) {
this(coder, Preconditions.checkNotNull(pipelineOptions).get().as(FlinkPipelineOptions.class).getFasterCopy());
}

public CoderTypeSerializer(Coder<T> coder, boolean fasterCopy) {
Preconditions.checkNotNull(coder);
Preconditions.checkNotNull(pipelineOptions);
this.coder = coder;
this.pipelineOptions = pipelineOptions;

FlinkPipelineOptions options = pipelineOptions.get().as(FlinkPipelineOptions.class);
this.fasterCopy = options.getFasterCopy();
this.fasterCopy = fasterCopy;
}

@Override
Expand All @@ -76,7 +69,7 @@ public boolean isImmutableType() {

@Override
public CoderTypeSerializer<T> duplicate() {
return new CoderTypeSerializer<>(coder, pipelineOptions);
return new CoderTypeSerializer<>(coder, fasterCopy);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,23 +47,21 @@ public class CoderTypeSerializer<T> extends TypeSerializer<T> {

private final Coder<T> coder;

/**
* {@link SerializablePipelineOptions} deserialization will cause {@link
* org.apache.beam.sdk.io.FileSystems} registration needed for {@link
* org.apache.beam.sdk.transforms.Reshuffle} translation.
*/
private final SerializablePipelineOptions pipelineOptions;

private final boolean fasterCopy;

public CoderTypeSerializer(Coder<T> coder, SerializablePipelineOptions pipelineOptions) {
this(
coder,
Preconditions.checkNotNull(pipelineOptions)
.get()
.as(FlinkPipelineOptions.class)
.getFasterCopy());
}

public CoderTypeSerializer(Coder<T> coder, boolean fasterCopy) {
Preconditions.checkNotNull(coder);
Preconditions.checkNotNull(pipelineOptions);
this.coder = coder;
this.pipelineOptions = pipelineOptions;

FlinkPipelineOptions options = pipelineOptions.get().as(FlinkPipelineOptions.class);
this.fasterCopy = options.getFasterCopy();
this.fasterCopy = fasterCopy;
}

@Override
Expand All @@ -73,7 +71,7 @@ public boolean isImmutableType() {

@Override
public CoderTypeSerializer<T> duplicate() {
return new CoderTypeSerializer<>(coder, pipelineOptions);
return new CoderTypeSerializer<>(coder, fasterCopy);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -237,6 +237,16 @@ public static StreamExecutionEnvironment createStreamExecutionEnvironment(
flinkStreamEnv.setParallelism(parallelism);
if (options.getMaxParallelism() > 0) {
flinkStreamEnv.setMaxParallelism(options.getMaxParallelism());
} else if (!options.isStreaming()) {
// In Flink maxParallelism defines the number of keyGroups.
// (see
// https://github.com/apache/flink/blob/e9dd4683f758b463d0b5ee18e49cecef6a70c5cf/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeAssignment.java#L76)
// The default value (parallelism * 1.5)
// (see
// https://github.com/apache/flink/blob/e9dd4683f758b463d0b5ee18e49cecef6a70c5cf/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeAssignment.java#L137-L147)
// create a lot of skew so we force maxParallelism = parallelism in Batch mode.
LOG.info("Setting maxParallelism to {}", parallelism);
flinkStreamEnv.setMaxParallelism(parallelism);
}
// set parallelism in the options (required by some execution code)
options.setParallelism(parallelism);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -246,7 +246,7 @@ public Long create(PipelineOptions options) {
if (options.as(StreamingOptions.class).isStreaming()) {
return 1000L;
} else {
return 1000000L;
return 5000L;
}
}
}
Expand Down Expand Up @@ -366,6 +366,13 @@ public Long create(PipelineOptions options) {

void setEnableStableInputDrain(Boolean enableStableInputDrain);

@Description(
"Set a slot sharing group for all bounded sources. This is required when using Datastream to have the same scheduling behaviour as the Dataset API.")
@Default.Boolean(true)
Boolean getForceSlotSharingGroup();

void setForceSlotSharingGroup(Boolean enableStableInputDrain);

static FlinkPipelineOptions defaults() {
return PipelineOptionsFactory.as(FlinkPipelineOptions.class);
}
Expand Down
Loading
Loading