Skip to content
Closed
Show file tree
Hide file tree
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 @@ -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 @@ -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