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

[flink-runner] Improve Datastream for batch performances #32440

Open
wants to merge 27 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
27 commits
Select commit Hold shift + click to select a range
b817957
[Flink] Set return type of bounded sources
jto Jul 23, 2024
ca9630a
[Flink] Use a lazy split enumerator for bounded sources
jto Jul 23, 2024
866fb78
[Flink] Default to maxParallelism = parallelism in batch
jto Aug 19, 2024
5c89f15
[Flink] Avoid re-serializing trigger on every element
jto Aug 20, 2024
f2b2eb7
[Flink] Avoid re-evaluating options every time a new state is stored
jto Aug 20, 2024
cf670c7
[Flink] Only serialize states namespace keys if necessary
jto Aug 21, 2024
e805ad1
[Flink] Make ToKeyedWorkItem part of the DoFnOperator
jto Aug 6, 2024
a96e8ed
[Flink] Remove ToBinaryKV
jto Aug 19, 2024
490576e
[Flink] Refactor CombinePerKeyTranslator
jto Aug 8, 2024
37847e7
[Flink] Combine before Reduce (no side-input only)
jto Aug 9, 2024
04f3d68
[Flink] Combine before GBK
jto Aug 23, 2024
f38fd11
[Flink] Combine before reduce (with side input)
jto Aug 28, 2024
d96a464
[Flink] Force slot sharing group in batch mode
jto Aug 27, 2024
d1de772
[Flink] Disable bundling in batch mode
jto Aug 26, 2024
4b205b7
[Flink] Lower default max bundle size in batch mode
jto Aug 23, 2024
e7699d0
[Flink] Code cleanup
jto Aug 28, 2024
ff9cb80
[Flink] fix WindowDoFnOperatorTest
jto Aug 29, 2024
6fddd1b
[Flink] spotless
jto Aug 30, 2024
7d62bf5
[Flink] fix broken tests
jto Sep 10, 2024
98a99f4
[Flink] Remove 1.14 compat code
jto Sep 12, 2024
21e83bd
[Flink] Fix flaky test
jto Sep 12, 2024
eaa08a5
[Flink] Use a custom key type to better distribute load
jto Oct 16, 2024
ac24d8f
[Flink] Add post commit triggers
jto Oct 16, 2024
a7b9023
[Flink] licence
jto Oct 16, 2024
ae0f3b4
[Flink] spotless
jto Oct 16, 2024
9738b01
Additional Flink github action trigger files
kennknowles Oct 18, 2024
db09fd5
Merge pull request #236 from kennknowles/flink-datastream-tests
jto Nov 5, 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
1 change: 1 addition & 0 deletions .github/trigger_files/beam_PostCommit_Go_VR_Flink.json
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"modification": 1,
"https://github.com/apache/beam/pull/32440": "testing datastream optimizations",
"https://github.com/apache/beam/pull/32648": "testing addition of Flink 1.19 support"
}
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
{
"https://github.com/apache/beam/pull/32648": "testing flink 1.19 support"
"https://github.com/apache/beam/pull/32440": "testing datastream optimizations",
"https://github.com/apache/beam/pull/32648": "testing flink 1.19 support"
}
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"modification": 1
"https://github.com/apache/beam/pull/32440": "test new datastream runner for batch"
}
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test"
"https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test",
"https://github.com/apache/beam/pull/32440": "test new datastream runner for batch"
}
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"modification": 1
"modification": 1,
"https://github.com/apache/beam/pull/32440": "test new datastream runner for batch"
}
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test",
"https://github.com/apache/beam/pull/32440": "testing datastream optimizations",
"https://github.com/apache/beam/pull/32648": "testing addition of Flink 1.19 support"
}
3 changes: 2 additions & 1 deletion .github/trigger_files/beam_PostCommit_Python.json
Original file line number Diff line number Diff line change
@@ -1,5 +1,6 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run.",
"modification": 4
"modification": 4,
"https://github.com/apache/beam/pull/32440": "test new datastream runner for batch"
}

1 change: 1 addition & 0 deletions .github/trigger_files/beam_PostCommit_XVR_Flink.json
Original file line number Diff line number Diff line change
@@ -1,3 +1,4 @@
{
"https://github.com/apache/beam/pull/32440": "testing datastream optimizations",
"https://github.com/apache/beam/pull/32648": "testing addition of Flink 1.19 support"
}
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 @@ -124,8 +127,7 @@ public void processElement(ProcessContext c) throws Exception {
key,
windowingStrategy,
ExecutableTriggerStateMachine.create(
TriggerStateMachines.stateMachineForTrigger(
TriggerTranslation.toProto(windowingStrategy.getTrigger()))),
TriggerStateMachines.stateMachineForTrigger(triggerProto)),
stateInternals,
timerInternals,
outputWindowedValue(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,23 +50,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 @@ -76,7 +74,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 @@ -262,7 +262,7 @@ public Long create(PipelineOptions options) {
if (options.as(StreamingOptions.class).isStreaming()) {
return 1000L;
} else {
return 1000000L;
return 5000L;
}
}
}
Expand Down Expand Up @@ -382,6 +382,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