Skip to content

Commit

Permalink
Revert "[Flink Runner] Add UseDataStreamForBatch option to Flink runn…
Browse files Browse the repository at this point in the history
…er to enable batch execution on DataStream API (apache#28614)"

This reverts commit 1f7e882.
  • Loading branch information
Abacn committed Jan 11, 2024
1 parent 9faf285 commit d4b321e
Show file tree
Hide file tree
Showing 19 changed files with 144 additions and 322 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.InternalTimeServiceManager;
import org.apache.flink.streaming.api.operators.InternalTimeServiceManagerImpl;
import org.apache.flink.streaming.api.operators.sorted.state.BatchExecutionInternalTimeServiceManager;

/** Compatibility layer for {@link AbstractStreamOperator} breaking changes. */
public abstract class AbstractStreamOperatorCompat<OutputT>
Expand All @@ -45,18 +44,9 @@ protected int numProcessingTimeTimers() {
return getTimeServiceManager()
.map(
manager -> {
InternalTimeServiceManager<?> tsm = getTimeServiceManagerCompat();
if (tsm instanceof InternalTimeServiceManagerImpl) {
final InternalTimeServiceManagerImpl<?> cast =
(InternalTimeServiceManagerImpl<?>) getTimeServiceManagerCompat();
return cast.numProcessingTimeTimers();
} else if (tsm instanceof BatchExecutionInternalTimeServiceManager) {
return 0;
} else {
throw new IllegalStateException(
String.format(
"Unknown implementation of InternalTimerServiceManager. %s", tsm));
}
final InternalTimeServiceManagerImpl<?> cast =
(InternalTimeServiceManagerImpl<?>) getTimeServiceManagerCompat();
return cast.numProcessingTimeTimers();
})
.orElse(0);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.InternalTimeServiceManager;
import org.apache.flink.streaming.api.operators.InternalTimeServiceManagerImpl;
import org.apache.flink.streaming.api.operators.sorted.state.BatchExecutionInternalTimeServiceManager;

/** Compatibility layer for {@link AbstractStreamOperator} breaking changes. */
public abstract class AbstractStreamOperatorCompat<OutputT>
Expand All @@ -45,18 +44,9 @@ protected int numProcessingTimeTimers() {
return getTimeServiceManager()
.map(
manager -> {
InternalTimeServiceManager<?> tsm = getTimeServiceManagerCompat();
if (tsm instanceof InternalTimeServiceManagerImpl) {
final InternalTimeServiceManagerImpl<?> cast =
(InternalTimeServiceManagerImpl<?>) getTimeServiceManagerCompat();
return cast.numProcessingTimeTimers();
} else if (tsm instanceof BatchExecutionInternalTimeServiceManager) {
return 0;
} else {
throw new IllegalStateException(
String.format(
"Unknown implementation of InternalTimerServiceManager. %s", tsm));
}
final InternalTimeServiceManagerImpl<?> cast =
(InternalTimeServiceManagerImpl<?>) getTimeServiceManagerCompat();
return cast.numProcessingTimeTimers();
})
.orElse(0);
}
Expand Down
8 changes: 0 additions & 8 deletions runners/flink/flink_runner.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -237,7 +237,6 @@ class ValidatesRunnerConfig {
String name
boolean streaming
boolean checkpointing
boolean useDataStreamForBatch
ArrayList<String> sickbayTests
}

Expand All @@ -256,7 +255,6 @@ def createValidatesRunnerTask(Map m) {
description = "Validates the ${runnerType} runner"
def pipelineOptionsArray = ["--runner=TestFlinkRunner",
"--streaming=${config.streaming}",
"--useDataStreamForBatch=${config.useDataStreamForBatch}",
"--parallelism=2",
]
if (config.checkpointing) {
Expand Down Expand Up @@ -318,17 +316,12 @@ def createValidatesRunnerTask(Map m) {
excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testFirstElementLate'
// https://github.com/apache/beam/issues/20844
excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testLateDataAccumulating'
if (!config.streaming) {
// FlinkBatchExecutionInternalTimeService does not support timer registration on timer firing.
excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimestampTests.testOnTimerTimestampSkew'
}
}
}
}
}

createValidatesRunnerTask(name: "validatesRunnerBatch", streaming: false, sickbayTests: sickbayTests)
createValidatesRunnerTask(name: "validatesRunnerBatchWithDataStream", streaming: false, useDataStreamForBatch: true, sickbayTests: sickbayTests)
createValidatesRunnerTask(name: "validatesRunnerStreaming", streaming: true, sickbayTests: sickbayTests)
// We specifically have a variant which runs with checkpointing enabled for the
// tests that require it since running a checkpoint variant is significantly
Expand All @@ -341,7 +334,6 @@ tasks.register('validatesRunner') {
group = 'Verification'
description "Validates Flink runner"
dependsOn validatesRunnerBatch
dependsOn validatesRunnerBatchWithDataStream
dependsOn validatesRunnerStreaming
dependsOn validatesRunnerStreamingCheckpointing
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import org.apache.beam.sdk.metrics.MetricsOptions;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.RuntimeExecutionMode;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.runtime.jobgraph.JobGraph;
Expand Down Expand Up @@ -102,17 +101,13 @@ public void translate(Pipeline pipeline) {
prepareFilesToStageForRemoteClusterExecution(options);

FlinkPipelineTranslator translator;
if (options.isStreaming() || options.getUseDataStreamForBatch()) {
if (options.isStreaming()) {
this.flinkStreamEnv = FlinkExecutionEnvironments.createStreamExecutionEnvironment(options);
if (hasUnboundedOutput && !flinkStreamEnv.getCheckpointConfig().isCheckpointingEnabled()) {
LOG.warn(
"UnboundedSources present which rely on checkpointing, but checkpointing is disabled.");
}
translator =
new FlinkStreamingPipelineTranslator(flinkStreamEnv, options, options.isStreaming());
if (!options.isStreaming()) {
flinkStreamEnv.setRuntimeMode(RuntimeExecutionMode.BATCH);
}
translator = new FlinkStreamingPipelineTranslator(flinkStreamEnv, options);
} else {
this.flinkBatchEnv = FlinkExecutionEnvironments.createBatchExecutionEnvironment(options);
translator = new FlinkBatchPipelineTranslator(flinkBatchEnv, options);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,11 +32,7 @@
* requiring flink on the classpath (e.g. to use with the direct runner).
*/
public interface FlinkPipelineOptions
extends PipelineOptions,
ApplicationNameOptions,
StreamingOptions,
FileStagingOptions,
VersionDependentFlinkPipelineOptions {
extends PipelineOptions, ApplicationNameOptions, StreamingOptions, FileStagingOptions {

String AUTO = "[auto]";
String PIPELINED = "PIPELINED";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,9 +81,8 @@ class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {

private int depth = 0;

public FlinkStreamingPipelineTranslator(
StreamExecutionEnvironment env, PipelineOptions options, boolean isStreaming) {
this.streamingContext = new FlinkStreamingTranslationContext(env, options, isStreaming);
public FlinkStreamingPipelineTranslator(StreamExecutionEnvironment env, PipelineOptions options) {
this.streamingContext = new FlinkStreamingTranslationContext(env, options);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,9 @@
import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
import org.apache.beam.runners.core.construction.SplittableParDo;
import org.apache.beam.runners.core.construction.TransformPayloadTranslatorRegistrar;
import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter;
import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows;
import org.apache.beam.runners.flink.translation.functions.ImpulseSourceFunction;
import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator;
import org.apache.beam.runners.flink.translation.wrappers.streaming.KvToByteBufferKeySelector;
Expand All @@ -52,9 +54,6 @@
import org.apache.beam.runners.flink.translation.wrappers.streaming.io.DedupingOperator;
import org.apache.beam.runners.flink.translation.wrappers.streaming.io.TestStreamSource;
import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper;
import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSource;
import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.bounded.FlinkBoundedSource;
import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.unbounded.FlinkUnboundedSource;
import org.apache.beam.sdk.coders.ByteArrayCoder;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.CoderException;
Expand Down Expand Up @@ -97,7 +96,6 @@
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps;
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.api.common.functions.RichFlatMapFunction;
import org.apache.flink.api.common.functions.RichMapFunction;
Expand Down Expand Up @@ -222,19 +220,16 @@ public void translateNode(
context.getExecutionEnvironment().getMaxParallelism() > 0
? context.getExecutionEnvironment().getMaxParallelism()
: context.getExecutionEnvironment().getParallelism();

FlinkUnboundedSource<T> unboundedSource =
FlinkSource.unbounded(
transform.getName(),
rawSource,
new SerializablePipelineOptions(context.getPipelineOptions()),
parallelism);
UnboundedSourceWrapper<T, ?> sourceWrapper =
new UnboundedSourceWrapper<>(
fullName, context.getPipelineOptions(), rawSource, parallelism);
nonDedupSource =
context
.getExecutionEnvironment()
.fromSource(
unboundedSource, WatermarkStrategy.noWatermarks(), fullName, withIdTypeInfo)
.uid(fullName);
.addSource(sourceWrapper)
.name(fullName)
.uid(fullName)
.returns(withIdTypeInfo);

if (rawSource.requiresDeduping()) {
source =
Expand Down Expand Up @@ -308,24 +303,15 @@ void translateNode(Impulse transform, FlinkStreamingTranslationContext context)
WindowedValue.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE),
context.getPipelineOptions());

FlinkBoundedSource<byte[]> impulseSource;
WatermarkStrategy<WindowedValue<byte[]>> watermarkStrategy;
if (context.isStreaming()) {
long shutdownAfterIdleSourcesMs =
context
.getPipelineOptions()
.as(FlinkPipelineOptions.class)
.getShutdownSourcesAfterIdleMs();
impulseSource = FlinkSource.unboundedImpulse(shutdownAfterIdleSourcesMs);
watermarkStrategy = WatermarkStrategy.forMonotonousTimestamps();
} else {
impulseSource = FlinkSource.boundedImpulse();
watermarkStrategy = WatermarkStrategy.noWatermarks();
}
long shutdownAfterIdleSourcesMs =
context
.getPipelineOptions()
.as(FlinkPipelineOptions.class)
.getShutdownSourcesAfterIdleMs();
SingleOutputStreamOperator<WindowedValue<byte[]>> source =
context
.getExecutionEnvironment()
.fromSource(impulseSource, watermarkStrategy, "Impulse")
.addSource(new ImpulseSourceFunction(shutdownAfterIdleSourcesMs), "Impulse")
.returns(typeInfo);

context.setOutputDataStream(context.getOutput(transform), source);
Expand All @@ -344,8 +330,7 @@ private static class ReadSourceTranslator<T>
@Override
void translateNode(
PTransform<PBegin, PCollection<T>> transform, FlinkStreamingTranslationContext context) {
if (ReadTranslation.sourceIsBounded(context.getCurrentTransform())
== PCollection.IsBounded.BOUNDED) {
if (context.getOutput(transform).isBounded().equals(PCollection.IsBounded.BOUNDED)) {
boundedTranslator.translateNode(transform, context);
} else {
unboundedTranslator.translateNode(transform, context);
Expand Down Expand Up @@ -376,26 +361,24 @@ public void translateNode(
}

String fullName = getCurrentTransformName(context);
int parallelism =
context.getExecutionEnvironment().getMaxParallelism() > 0
? context.getExecutionEnvironment().getMaxParallelism()
: context.getExecutionEnvironment().getParallelism();

FlinkBoundedSource<T> flinkBoundedSource =
FlinkSource.bounded(
transform.getName(),
rawSource,
new SerializablePipelineOptions(context.getPipelineOptions()),
parallelism);

UnboundedSource<T, ?> adaptedRawSource = new BoundedToUnboundedSourceAdapter<>(rawSource);
DataStream<WindowedValue<T>> source;
try {
int parallelism =
context.getExecutionEnvironment().getMaxParallelism() > 0
? context.getExecutionEnvironment().getMaxParallelism()
: context.getExecutionEnvironment().getParallelism();
UnboundedSourceWrapperNoValueWithRecordId<T, ?> sourceWrapper =
new UnboundedSourceWrapperNoValueWithRecordId<>(
new UnboundedSourceWrapper<>(
fullName, context.getPipelineOptions(), adaptedRawSource, parallelism));
source =
context
.getExecutionEnvironment()
.fromSource(
flinkBoundedSource, WatermarkStrategy.noWatermarks(), fullName, outputTypeInfo)
.uid(fullName);
.addSource(sourceWrapper)
.name(fullName)
.uid(fullName)
.returns(outputTypeInfo);
} catch (Exception e) {
throw new RuntimeException("Error while translating BoundedSource: " + rawSource, e);
}
Expand Down Expand Up @@ -562,9 +545,7 @@ static <InputT, OutputT> void translateParDo(
KeySelector<WindowedValue<InputT>, ?> keySelector = null;
boolean stateful = false;
DoFnSignature signature = DoFnSignatures.getSignature(doFn.getClass());
if (!signature.stateDeclarations().isEmpty()
|| !signature.timerDeclarations().isEmpty()
|| !signature.timerFamilyDeclarations().isEmpty()) {
if (signature.stateDeclarations().size() > 0 || signature.timerDeclarations().size() > 0) {
// Based on the fact that the signature is stateful, DoFnSignatures ensures
// that it is also keyed
keyCoder = ((KvCoder) input.getCoder()).getKeyCoder();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,6 @@ class FlinkStreamingTranslationContext {

private final StreamExecutionEnvironment env;
private final PipelineOptions options;
private final boolean isStreaming;

/**
* Keeps a mapping between the output value of the PTransform and the Flink Operator that produced
Expand All @@ -63,11 +62,9 @@ class FlinkStreamingTranslationContext {

private AppliedPTransform<?, ?, ?> currentTransform;

public FlinkStreamingTranslationContext(
StreamExecutionEnvironment env, PipelineOptions options, boolean isStreaming) {
public FlinkStreamingTranslationContext(StreamExecutionEnvironment env, PipelineOptions options) {
this.env = checkNotNull(env);
this.options = checkNotNull(options);
this.isStreaming = isStreaming;
}

public StreamExecutionEnvironment getExecutionEnvironment() {
Expand All @@ -78,10 +75,6 @@ public PipelineOptions getPipelineOptions() {
return options;
}

public boolean isStreaming() {
return isStreaming;
}

@SuppressWarnings("unchecked")
public <T> DataStream<T> getInputDataStream(PValue value) {
return (DataStream<T>) dataStreams.get(value);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
class FlinkTransformOverrides {
static List<PTransformOverride> getDefaultOverrides(FlinkPipelineOptions options) {
ImmutableList.Builder<PTransformOverride> builder = ImmutableList.builder();
if (options.isStreaming() || options.getUseDataStreamForBatch()) {
if (options.isStreaming()) {
builder
.add(
PTransformOverride.of(
Expand Down

This file was deleted.

0 comments on commit d4b321e

Please sign in to comment.