Skip to content
Closed
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 @@ -20,6 +20,8 @@
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.connector.sink2.Sink;
import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink;
import org.apache.flink.api.connector.source.Boundedness;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.cdc.common.annotation.Internal;
import org.apache.flink.cdc.common.annotation.VisibleForTesting;
import org.apache.flink.cdc.common.configuration.Configuration;
Expand Down Expand Up @@ -48,6 +50,8 @@
import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
import org.apache.flink.streaming.api.transformations.LegacySinkTransformation;
import org.apache.flink.streaming.api.transformations.PhysicalTransformation;
import org.apache.flink.streaming.api.transformations.WithBoundedness;
import org.apache.flink.util.Preconditions;

import java.lang.reflect.InvocationTargetException;

Expand Down Expand Up @@ -158,8 +162,9 @@ private <CommT> void addCommittingTopology(
((WithPreCommitTopology<Event, CommT>) sink).addPreCommitTopology(written);
}

// TODO: Hard coding stream mode and checkpoint
boolean isBatchMode = false;
boolean isBatchMode = !existsUnboundedSource(inputStream.getTransformation());

// TODO: Hard coding checkpoint
boolean isCheckpointingEnabled = true;
DataStream<CommittableMessage<CommT>> committed =
preCommitted.transform(
Expand Down Expand Up @@ -209,4 +214,16 @@ private static <CommT> SimpleVersionedSerializer<CommT> getCommittableSerializer
throw new RuntimeException("Failed to create CommitterOperatorFactory", e);
}
}

private boolean existsUnboundedSource(final Transformation<?> transformation) {
return isUnboundedSource(transformation)
|| transformation.getTransitivePredecessors().stream()
.anyMatch(this::isUnboundedSource);
}

private boolean isUnboundedSource(final Transformation<?> transformation) {
Preconditions.checkNotNull(transformation);
return transformation instanceof WithBoundedness
&& ((WithBoundedness) transformation).getBoundedness() != Boundedness.BOUNDED;
}
}
Loading