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 @@ -141,7 +141,7 @@ static ExecutionEnvironment createBatchExecutionEnvironment(
}
}

// Set the correct parallelism, required by UnboundedSourceWrapper to generate consistent
// Set the correct parallelism, required by FlinkUnboundedSource to generate consistent
// splits.
final int parallelism;
if (flinkBatchEnv instanceof CollectionEnvironment) {
Expand Down Expand Up @@ -230,7 +230,7 @@ public static StreamExecutionEnvironment createStreamExecutionEnvironment(
LOG.info("Using Flink Master URL {}:{}.", hostAndPort.getHost(), hostAndPort.getPort());
}

// Set the parallelism, required by UnboundedSourceWrapper to generate consistent splits.
// Set the parallelism, required by FlinkUnboundedSource to generate consistent splits.
final int parallelism =
determineParallelism(
options.getParallelism(), flinkStreamEnv.getParallelism(), flinkConfiguration);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,8 @@
import org.apache.beam.runners.flink.translation.wrappers.streaming.io.DedupingOperator;
import org.apache.beam.runners.flink.translation.wrappers.streaming.io.StreamingImpulseSource;
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.unbounded.FlinkUnboundedSource;
import org.apache.beam.runners.fnexecution.control.SdkHarnessClient;
import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
import org.apache.beam.runners.fnexecution.wire.WireCoders;
Expand Down Expand Up @@ -108,6 +109,7 @@
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets;
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.api.common.typeinfo.TypeInformation;
Expand Down Expand Up @@ -601,15 +603,17 @@ private static <T> DataStream<WindowedValue<T>> translateUnboundedSource(

int parallelism =
env.getMaxParallelism() > 0 ? env.getMaxParallelism() : env.getParallelism();
UnboundedSourceWrapper<T, ?> sourceWrapper =
new UnboundedSourceWrapper<>(
transformName, pipelineOptions, unboundedSource, parallelism);
FlinkUnboundedSource<T> sourceWrapper =
FlinkSource.unbounded(
transformName,
unboundedSource,
new SerializablePipelineOptions(pipelineOptions),
parallelism);
nonDedupSource =
env.addSource(sourceWrapper)
.name(transformName)
.uid(transformName)
.returns(withIdTypeInfo);

env
.fromSource(
sourceWrapper, WatermarkStrategy.noWatermarks(), transformName, withIdTypeInfo)
.uid(transformName);
if (unboundedSource.requiresDeduping()) {
source =
nonDedupSource
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,6 @@
import org.apache.beam.runners.flink.translation.wrappers.streaming.io.BeamStoppableFunction;
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;
Expand Down Expand Up @@ -1508,118 +1507,4 @@ void translateNode(TestStream<T> testStream, FlinkStreamingTranslationContext co
}
}

/**
* Wrapper for {@link UnboundedSourceWrapper}, which simplifies output type, namely, removes
* {@link ValueWithRecordId}.
*/
static class UnboundedSourceWrapperNoValueWithRecordId<
OutputT, CheckpointMarkT extends UnboundedSource.CheckpointMark>
extends RichParallelSourceFunction<WindowedValue<OutputT>>
implements BeamStoppableFunction,
CheckpointListener,
CheckpointedFunction,
ProcessingTimeCallback {

private final UnboundedSourceWrapper<OutputT, CheckpointMarkT> unboundedSourceWrapper;

@VisibleForTesting
UnboundedSourceWrapper<OutputT, CheckpointMarkT> getUnderlyingSource() {
return unboundedSourceWrapper;
}

UnboundedSourceWrapperNoValueWithRecordId(
UnboundedSourceWrapper<OutputT, CheckpointMarkT> unboundedSourceWrapper) {
this.unboundedSourceWrapper = unboundedSourceWrapper;
}

@Override
public void open(Configuration parameters) throws Exception {
unboundedSourceWrapper.setRuntimeContext(getRuntimeContext());
unboundedSourceWrapper.open(parameters);
}

@Override
public void run(SourceContext<WindowedValue<OutputT>> ctx) throws Exception {
unboundedSourceWrapper.run(new SourceContextWrapper(ctx));
}

@Override
public void initializeState(FunctionInitializationContext context) throws Exception {
unboundedSourceWrapper.initializeState(context);
}

@Override
public void snapshotState(FunctionSnapshotContext context) throws Exception {
unboundedSourceWrapper.snapshotState(context);
}

@Override
public void notifyCheckpointComplete(long checkpointId) throws Exception {
unboundedSourceWrapper.notifyCheckpointComplete(checkpointId);
}

@Override
public void stop() {
unboundedSourceWrapper.stop();
}

@Override
public void cancel() {
unboundedSourceWrapper.cancel();
}

@Override
public void onProcessingTime(long timestamp) throws Exception {
unboundedSourceWrapper.onProcessingTime(timestamp);
}

private final class SourceContextWrapper
implements SourceContext<WindowedValue<ValueWithRecordId<OutputT>>> {

private final SourceContext<WindowedValue<OutputT>> ctx;

private SourceContextWrapper(SourceContext<WindowedValue<OutputT>> ctx) {
this.ctx = ctx;
}

@Override
public void collect(WindowedValue<ValueWithRecordId<OutputT>> element) {
OutputT originalValue = element.getValue().getValue();
WindowedValue<OutputT> output =
WindowedValue.of(
originalValue, element.getTimestamp(), element.getWindows(), element.getPane());
ctx.collect(output);
}

@Override
public void collectWithTimestamp(
WindowedValue<ValueWithRecordId<OutputT>> element, long timestamp) {
OutputT originalValue = element.getValue().getValue();
WindowedValue<OutputT> output =
WindowedValue.of(
originalValue, element.getTimestamp(), element.getWindows(), element.getPane());
ctx.collectWithTimestamp(output, timestamp);
}

@Override
public void emitWatermark(Watermark mark) {
ctx.emitWatermark(mark);
}

@Override
public void markAsTemporarilyIdle() {
ctx.markAsTemporarilyIdle();
}

@Override
public Object getCheckpointLock() {
return ctx.getCheckpointLock();
}

@Override
public void close() {
ctx.close();
}
}
}
}
Loading