diff --git a/build-with-java17.sh b/build-with-java17.sh new file mode 100755 index 00000000000..40ea89b6ec8 --- /dev/null +++ b/build-with-java17.sh @@ -0,0 +1,51 @@ +#!/usr/bin/env bash +# +# Build full flink-cdc with Java 17 (sdkman), producing both compat-flink1 and compat-flink2 artifacts. +# Usage: ./build-with-java17.sh [skip-tests] +# skip-tests: pass any non-empty argument to skip tests (e.g. ./build-with-java17.sh 1) +# + +set -e + +# Use Java 17 via sdkman (no-op if already Java 17) +if [[ -n "$SDKMAN_DIR" ]]; then + echo "Using sdkman: switching to Java 17..." + source "${SDKMAN_DIR}/bin/sdkman-init.sh" + # Prefer Temurin 17; adjust to your installed version (sdk list java) + sdk use java 17.0.18-tem 2>/dev/null || sdk use java 17.0.11-tem 2>/dev/null || sdk use java 17.0.9-tem 2>/dev/null || true +fi + +if [[ -z "$JAVA_HOME" ]] || ! java -version 2>&1 | grep -q 'version "17'; then + echo "WARNING: Java 17 not detected. Set JAVA_HOME or run: sdk use java 17.x.x-tem" + java -version 2>&1 || true +fi + +MVN="${MVN:-$HOME/apache-maven-3.8.9/bin/mvn}" +if [[ ! -x "$MVN" ]]; then + MVN="mvn" +fi + +SKIP_TESTS="" +if [[ -n "$1" ]]; then + SKIP_TESTS="-DskipTests -Drat.skip=true" + echo "Build will skip tests (and RAT)." +fi + +# Java 17 overrides for default build (Flink 1.20 + compat-flink1) +JAVA17="-Djava.version=17 -Dsource.java.version=17 -Dtarget.java.version=17" + +echo "==========================================" +echo "Step 1: Full build (Flink 1.20, compat-flink1, Java 17)" +echo "==========================================" +"$MVN" clean install $JAVA17 $SKIP_TESTS -q + +echo "" +echo "Step 2: Build Flink 2.2 compat and dist (no clean, keeps 1.20 artifacts)" +echo "==========================================" +"$MVN" install -Pflink-2.2 -pl flink-cdc-flink-compat/flink-cdc-flink-compat-flink2,flink-cdc-dist $SKIP_TESTS -q + +echo "" +echo "Done. Key artifacts:" +echo " Flink 1.20: flink-cdc-dist/target/flink-cdc-dist-*-1.20.jar" +echo " Flink 2.2: flink-cdc-dist/target/flink-cdc-dist-*-2.2.jar" +ls -la flink-cdc-dist/target/flink-cdc-dist-*.jar 2>/dev/null || true diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/exceptions/ValidationException.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/exceptions/ValidationException.java new file mode 100644 index 00000000000..adfbbfa1689 --- /dev/null +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/exceptions/ValidationException.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.common.exceptions; + +import org.apache.flink.cdc.common.annotation.PublicEvolving; + +import javax.annotation.Nullable; + +/** + * Exception for validation errors (e.g. invalid options, unsupported options). Defined in + * flink-cdc-common to avoid runtime dependency on Flink table API (e.g. {@code + * org.apache.flink.table.api.ValidationException}), which may not be on the classpath when running + * with different Flink versions. + */ +@PublicEvolving +public class ValidationException extends RuntimeException { + + public ValidationException(String message) { + super(message); + } + + public ValidationException(String message, @Nullable Throwable cause) { + super(message, cause); + } +} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/factories/FactoryHelper.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/factories/FactoryHelper.java index 8e4ff02276e..f9b53c97347 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/factories/FactoryHelper.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/factories/FactoryHelper.java @@ -21,9 +21,9 @@ import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.configuration.FallbackKey; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.utils.Preconditions; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.ValidationException; import java.util.Arrays; import java.util.HashMap; diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/EventSinkProvider.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/EventSinkProvider.java index ec1d34b677b..8f062114267 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/EventSinkProvider.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/EventSinkProvider.java @@ -19,12 +19,11 @@ import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.cdc.common.annotation.PublicEvolving; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; /** * A marker interface used to provide an event sink for writing change events to external systems. - * We can reuse exiting Flink {@link Sink} and Flink {@link SinkFunction} implementation, and we can - * support our own {@code EventSink} Implementation in the future. + * We can reuse existing Flink {@link Sink} implementations; legacy SinkFunction-based providers are + * supported via the Flink 1.x compat module (flink-cdc-flink-compat-flink1). */ @PublicEvolving public interface EventSinkProvider {} diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/source/EventSourceProvider.java b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/source/EventSourceProvider.java index bf33f839e74..07294210ab7 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/source/EventSourceProvider.java +++ b/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/source/EventSourceProvider.java @@ -19,12 +19,11 @@ import org.apache.flink.api.connector.source.Source; import org.apache.flink.cdc.common.annotation.PublicEvolving; -import org.apache.flink.streaming.api.functions.source.SourceFunction; /** * A marker interface used to provide an event source for reading events from external systems. We - * can reuse exiting Flink {@link Source} and Flink {@link SourceFunction} implementation, and we - * can support our own {@code EventSource} implementation in the future. + * can reuse existing Flink {@link Source} implementations; legacy SourceFunction-based providers + * are supported via the Flink 1.x compat module (flink-cdc-flink-compat-flink1). */ @PublicEvolving public interface EventSourceProvider {} diff --git a/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/factories/FactoryHelperTests.java b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/factories/FactoryHelperTests.java index a7be9c3f5e7..d04d958a9a1 100644 --- a/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/factories/FactoryHelperTests.java +++ b/flink-cdc-common/src/test/java/org/apache/flink/cdc/common/factories/FactoryHelperTests.java @@ -20,7 +20,7 @@ import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.ConfigOptions; import org.apache.flink.cdc.common.configuration.Configuration; -import org.apache.flink.table.api.ValidationException; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.shaded.guava31.com.google.common.collect.Sets; diff --git a/flink-cdc-composer/pom.xml b/flink-cdc-composer/pom.xml index b5f1d8a187a..e88c686ebcd 100644 --- a/flink-cdc-composer/pom.xml +++ b/flink-cdc-composer/pom.xml @@ -33,11 +33,22 @@ limitations under the License. flink-cdc-common ${project.version} + + org.apache.flink + flink-cdc-flink-compat-api + ${project.version} + org.apache.flink flink-cdc-runtime ${project.version} + + org.apache.flink + flink-cdc-flink-compat-flink1 + ${project.version} + test + org.apache.flink flink-cdc-pipeline-connector-values diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/compat/FlinkPipelineBridges.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/compat/FlinkPipelineBridges.java new file mode 100644 index 00000000000..4c9dee1d765 --- /dev/null +++ b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/compat/FlinkPipelineBridges.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.composer.flink.compat; + +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.flink.compat.FlinkPipelineBridge; + +import java.util.Iterator; +import java.util.ServiceLoader; + +/** Loads the {@link FlinkPipelineBridge} implementation via {@link ServiceLoader}. */ +@Internal +public final class FlinkPipelineBridges { + + /** + * Returns the first available {@link FlinkPipelineBridge} implementation. + * + *

Exactly one compat JAR (flink-cdc-flink-compat-flink1 or flink-cdc-flink-compat-flink2) + * must be on the classpath and provide a service implementation. + */ + public static FlinkPipelineBridge getDefault() { + ServiceLoader loader = + ServiceLoader.load( + FlinkPipelineBridge.class, FlinkPipelineBridge.class.getClassLoader()); + Iterator it = loader.iterator(); + if (!it.hasNext()) { + throw new IllegalStateException( + "No FlinkPipelineBridge implementation found. Add flink-cdc-flink-compat-flink1 " + + "or flink-cdc-flink-compat-flink2 to the classpath."); + } + FlinkPipelineBridge bridge = it.next(); + if (it.hasNext()) { + throw new IllegalStateException( + "Multiple FlinkPipelineBridge implementations found. Only one compat JAR " + + "(flink-cdc-flink-compat-flink1 or flink-cdc-flink-compat-flink2) should be on the classpath."); + } + return bridge; + } + + private FlinkPipelineBridges() {} +} diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslator.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslator.java index aa530300b9e..13c590ae65b 100644 --- a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslator.java +++ b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslator.java @@ -19,7 +19,6 @@ 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.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.annotation.VisibleForTesting; import org.apache.flink.cdc.common.configuration.Configuration; @@ -28,30 +27,24 @@ import org.apache.flink.cdc.common.factories.FactoryHelper; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.common.sink.EventSinkProvider; -import org.apache.flink.cdc.common.sink.FlinkSinkFunctionProvider; import org.apache.flink.cdc.common.sink.FlinkSinkProvider; import org.apache.flink.cdc.composer.definition.SinkDef; import org.apache.flink.cdc.composer.flink.FlinkEnvironmentUtils; +import org.apache.flink.cdc.composer.flink.compat.FlinkPipelineBridges; import org.apache.flink.cdc.composer.utils.FactoryDiscoveryUtils; -import org.apache.flink.cdc.runtime.operators.sink.BatchDataSinkFunctionOperator; -import org.apache.flink.cdc.runtime.operators.sink.DataSinkFunctionOperator; +import org.apache.flink.cdc.flink.compat.FlinkPipelineBridge; import org.apache.flink.cdc.runtime.operators.sink.DataSinkWriterOperatorFactory; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; import org.apache.flink.streaming.api.connector.sink2.CommittableMessageTypeInfo; -import org.apache.flink.streaming.api.connector.sink2.WithPostCommitTopology; -import org.apache.flink.streaming.api.connector.sink2.WithPreCommitTopology; -import org.apache.flink.streaming.api.connector.sink2.WithPreWriteTopology; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.StreamSink; -import org.apache.flink.streaming.api.transformations.LegacySinkTransformation; -import org.apache.flink.streaming.api.transformations.PhysicalTransformation; +import java.lang.reflect.InvocationHandler; import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; /** Translator used to build {@link DataSink} for given {@link DataStream}. */ @Internal @@ -96,26 +89,25 @@ public void translate( boolean isBatchMode, OperatorID schemaOperatorID, OperatorUidGenerator operatorUidGenerator) { - // Get sink provider EventSinkProvider eventSinkProvider = dataSink.getEventSinkProvider(); + if (eventSinkProvider == null) { + return; + } String sinkName = generateSinkName(sinkDef); if (eventSinkProvider instanceof FlinkSinkProvider) { - // Sink V2 FlinkSinkProvider sinkProvider = (FlinkSinkProvider) eventSinkProvider; Sink sink = sinkProvider.getSink(); sinkTo(input, sink, sinkName, isBatchMode, schemaOperatorID, operatorUidGenerator); - } else if (eventSinkProvider instanceof FlinkSinkFunctionProvider) { - // SinkFunction - FlinkSinkFunctionProvider sinkFunctionProvider = - (FlinkSinkFunctionProvider) eventSinkProvider; - SinkFunction sinkFunction = sinkFunctionProvider.getSinkFunction(); - sinkTo( + } else { + // SinkFunction path (Flink 1.x compat) + FlinkPipelineBridge bridge = FlinkPipelineBridges.getDefault(); + bridge.sinkTo( input, - sinkFunction, + eventSinkProvider, sinkName, isBatchMode, schemaOperatorID, - operatorUidGenerator); + operatorUidGenerator::generateUid); } } @@ -128,12 +120,11 @@ void sinkTo( OperatorID schemaOperatorID, OperatorUidGenerator operatorUidGenerator) { DataStream stream = input; - // Pre-write topology - if (sink instanceof WithPreWriteTopology) { - stream = ((WithPreWriteTopology) sink).addPreWriteTopology(stream); - } + // Pre-write topology (reflection for Flink 1.x WithPreWriteTopology vs 2.x + // SupportsPreWriteTopology) + stream = addPreWriteTopologyIfSupported(sink, stream); - if (sink instanceof TwoPhaseCommittingSink) { + if (supportsTwoPhaseCommit(sink)) { addCommittingTopology( sink, stream, sinkName, isBatchMode, schemaOperatorID, operatorUidGenerator); } else { @@ -146,29 +137,51 @@ void sinkTo( } } - private void sinkTo( - DataStream input, - SinkFunction sinkFunction, - String sinkName, - boolean isBatchMode, - OperatorID schemaOperatorID, - OperatorUidGenerator operatorUidGenerator) { - StreamSink sinkOperator; - if (isBatchMode) { - sinkOperator = new BatchDataSinkFunctionOperator(sinkFunction); - } else { - sinkOperator = new DataSinkFunctionOperator(sinkFunction, schemaOperatorID); + /** + * True if sink has createCommitter and getCommittableSerializer (Flink 1.x + * TwoPhaseCommittingSink vs 2.x SupportsCommitter). + * + *

We must also consider methods declared on superclasses (for example {@code + * PaimonEventSink} extends {@code PaimonSink} which declares {@code createCommitter}). Using + * {@code getDeclaredMethods()} only on the concrete class would miss those and incorrectly + * disable the committer stage. + */ + private static boolean supportsTwoPhaseCommit(Sink sink) { + try { + boolean hasCreateCommitter = false; + boolean hasGetCommittableSerializer = false; + + // getMethods() returns all public methods, including those from superclasses and + // interfaces, without forcing us to load 1.x-only types such as Sink.InitContext. + for (Method m : sink.getClass().getMethods()) { + if ("createCommitter".equals(m.getName()) && m.getParameterCount() == 0) { + hasCreateCommitter = true; + } + if ("getCommittableSerializer".equals(m.getName()) && m.getParameterCount() == 0) { + hasGetCommittableSerializer = true; + } + } + return hasCreateCommitter && hasGetCommittableSerializer; + } catch (NoClassDefFoundError e) { + // Sink class references 1.x-only types (e.g. Sink.InitContext); assume two-phase + return true; + } + } + + /** + * Calls addPreWriteTopology via reflection to support both Flink 1.x (WithPreWriteTopology) and + * 2.x (SupportsPreWriteTopology). + */ + private static DataStream addPreWriteTopologyIfSupported( + Sink sink, DataStream stream) { + try { + Method m = sink.getClass().getMethod("addPreWriteTopology", DataStream.class); + @SuppressWarnings("unchecked") + DataStream result = (DataStream) m.invoke(sink, stream); + return result != null ? result : stream; + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { + return stream; } - final StreamExecutionEnvironment executionEnvironment = input.getExecutionEnvironment(); - PhysicalTransformation transformation = - new LegacySinkTransformation<>( - input.getTransformation(), - SINK_WRITER_PREFIX + sinkName, - sinkOperator, - executionEnvironment.getParallelism(), - false); - transformation.setUid(operatorUidGenerator.generateUid("sink-writer")); - executionEnvironment.addOperator(transformation); } private void addCommittingTopology( @@ -189,11 +202,8 @@ private void addCommittingTopology( sink, isBatchMode, schemaOperatorID)) .uid(operatorUidGenerator.generateUid("sink-writer")); - DataStream> preCommitted = written; - if (sink instanceof WithPreCommitTopology) { - preCommitted = - ((WithPreCommitTopology) sink).addPreCommitTopology(written); - } + DataStream> preCommitted = + addPreCommitTopologyIfSupported(sink, written); // TODO: Hard coding checkpoint boolean isCheckpointingEnabled = true; @@ -206,8 +216,36 @@ private void addCommittingTopology( sink, isBatchMode, isCheckpointingEnabled)) .uid(operatorUidGenerator.generateUid("sink-committer")); - if (sink instanceof WithPostCommitTopology) { - ((WithPostCommitTopology) sink).addPostCommitTopology(committed); + addPostCommitTopologyIfSupported(sink, committed); + } + + /** + * Calls addPreCommitTopology via reflection (Flink 1.x WithPreCommitTopology vs 2.x + * SupportsPreCommitTopology). + */ + @SuppressWarnings("unchecked") + private static DataStream> addPreCommitTopologyIfSupported( + Sink sink, DataStream> written) { + try { + Method m = sink.getClass().getMethod("addPreCommitTopology", DataStream.class); + Object result = m.invoke(sink, written); + return result != null ? (DataStream>) result : written; + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { + return written; + } + } + + /** + * Calls addPostCommitTopology via reflection (Flink 1.x WithPostCommitTopology vs 2.x + * SupportsPostCommitTopology). + */ + private static void addPostCommitTopologyIfSupported( + Sink sink, DataStream> committed) { + try { + Method m = sink.getClass().getMethod("addPostCommitTopology", DataStream.class); + m.invoke(sink, committed); + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { + // Sink does not support post-commit topology } } @@ -217,8 +255,9 @@ private String generateSinkName(SinkDef sinkDef) { } private static SimpleVersionedSerializer getCommittableSerializer(Object sink) { - // FIX ME: TwoPhaseCommittingSink has been deprecated, and its signature has changed - // during Flink 1.18 to 1.19. Remove this when Flink 1.18 is no longer supported. + // Uses reflection-friendly method name (works with TwoPhaseCommittingSink and + // SupportsCommitter) during Flink 1.18 to 1.19. Remove this when Flink 1.18 is no longer + // supported. try { return (SimpleVersionedSerializer) sink.getClass().getDeclaredMethod("getCommittableSerializer").invoke(sink); @@ -227,19 +266,37 @@ private static SimpleVersionedSerializer getCommittableSerializer } } + /** + * Creates a {@code CommitterOperatorFactory} via reflection. + * + *

Flink 1.19+ / 2.x requires the first constructor parameter to be {@code + * SupportsCommitter}. Sinks like PaimonSink declare {@code createCommitter()} (no-arg) + * and {@code getCommittableSerializer()} without implementing the {@code SupportsCommitter} + * interface. In that case we create a dynamic proxy that adapts the sink. + */ + @SuppressWarnings("unchecked") private static OneInputStreamOperatorFactory, CommittableMessage> getCommitterOperatorFactory( Sink sink, boolean isBatchMode, boolean isCheckpointingEnabled) { - // FIX ME: OneInputStreamOperatorFactory is an @Internal class, and its signature has - // changed during Flink 1.18 to 1.19. Remove this when Flink 1.18 is no longer supported. try { + Class factoryClass = + Class.forName( + "org.apache.flink.streaming.runtime.operators.sink.CommitterOperatorFactory"); + java.lang.reflect.Constructor ctor = factoryClass.getDeclaredConstructors()[0]; + Class firstParamType = ctor.getParameterTypes()[0]; + + Object firstArg; + if (firstParamType.isInstance(sink)) { + firstArg = sink; + } else { + firstArg = adaptToSupportsCommitter(sink, firstParamType); + } + + ctor.setAccessible(true); return (OneInputStreamOperatorFactory< CommittableMessage, CommittableMessage>) - Class.forName( - "org.apache.flink.streaming.runtime.operators.sink.CommitterOperatorFactory") - .getDeclaredConstructors()[0] - .newInstance(sink, isBatchMode, isCheckpointingEnabled); + ctor.newInstance(firstArg, isBatchMode, isCheckpointingEnabled); } catch (ClassNotFoundException | InstantiationException @@ -248,4 +305,55 @@ private static SimpleVersionedSerializer getCommittableSerializer throw new RuntimeException("Failed to create CommitterOperatorFactory", e); } } + + /** + * Creates a dynamic proxy implementing {@code SupportsCommitter} (or whatever the target + * interface is) by delegating to the sink's existing {@code createCommitter()} and {@code + * getCommittableSerializer()} methods. + */ + private static Object adaptToSupportsCommitter(Sink sink, Class targetInterface) { + return java.lang.reflect.Proxy.newProxyInstance( + targetInterface.getClassLoader(), + new Class[] {targetInterface}, + new SupportsCommitterInvocationHandler(sink)); + } + + /** Serializable invocation handler used to adapt sinks to SupportsCommitter. */ + private static final class SupportsCommitterInvocationHandler + implements InvocationHandler, java.io.Serializable { + + private static final long serialVersionUID = 1L; + + private final Sink sink; + + private SupportsCommitterInvocationHandler(Sink sink) { + this.sink = sink; + } + + @Override + public Object invoke(Object proxy, Method method, Object[] args) throws Throwable { + String name = method.getName(); + if ("createCommitter".equals(name)) { + // SupportsCommitter.createCommitter(CommitterInitContext) -> + // delegate to sink.createCommitter() (no-arg) + Method noArgMethod = sink.getClass().getMethod("createCommitter"); + noArgMethod.setAccessible(true); + return noArgMethod.invoke(sink); + } + if ("getCommittableSerializer".equals(name)) { + Method serMethod = sink.getClass().getMethod("getCommittableSerializer"); + serMethod.setAccessible(true); + return serMethod.invoke(sink); + } + // Delegate any other method (toString, equals, hashCode) to sink when present + try { + Method sinkMethod = sink.getClass().getMethod(name, method.getParameterTypes()); + sinkMethod.setAccessible(true); + return sinkMethod.invoke(sink, args); + } catch (NoSuchMethodException ignore) { + // Fall back to default behaviour for Object methods on proxy + return method.invoke(this, args); + } + } + } } diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSourceTranslator.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSourceTranslator.java index e4f684f8a37..5ad964762b7 100644 --- a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSourceTranslator.java +++ b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/DataSourceTranslator.java @@ -17,7 +17,6 @@ package org.apache.flink.cdc.composer.flink.translator; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.event.Event; @@ -25,12 +24,11 @@ import org.apache.flink.cdc.common.factories.FactoryHelper; import org.apache.flink.cdc.common.source.DataSource; import org.apache.flink.cdc.common.source.EventSourceProvider; -import org.apache.flink.cdc.common.source.FlinkSourceFunctionProvider; -import org.apache.flink.cdc.common.source.FlinkSourceProvider; import org.apache.flink.cdc.composer.definition.SourceDef; import org.apache.flink.cdc.composer.flink.FlinkEnvironmentUtils; +import org.apache.flink.cdc.composer.flink.compat.FlinkPipelineBridges; import org.apache.flink.cdc.composer.utils.FactoryDiscoveryUtils; -import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo; +import org.apache.flink.cdc.flink.compat.FlinkPipelineBridge; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -45,45 +43,15 @@ public DataStream translate( StreamExecutionEnvironment env, int sourceParallelism, OperatorUidGenerator operatorUidGenerator) { - // Get source provider EventSourceProvider eventSourceProvider = dataSource.getEventSourceProvider(); - - return createDataStreamSource(env, eventSourceProvider, sourceDef) - .setParallelism(sourceParallelism) + String sourceName = sourceDef.getName().orElse(generateDefaultSourceName(sourceDef)); + FlinkPipelineBridge bridge = FlinkPipelineBridges.getDefault(); + DataStreamSource stream = + bridge.createDataStreamSource(env, eventSourceProvider, sourceName); + return stream.setParallelism(sourceParallelism) .uid(operatorUidGenerator.generateUid("source")); } - private DataStreamSource createDataStreamSource( - StreamExecutionEnvironment env, - EventSourceProvider eventSourceProvider, - SourceDef sourceDef) { - // Source - if (eventSourceProvider instanceof FlinkSourceProvider) { - FlinkSourceProvider sourceProvider = (FlinkSourceProvider) eventSourceProvider; - return env.fromSource( - sourceProvider.getSource(), - WatermarkStrategy.noWatermarks(), - sourceDef.getName().orElse(generateDefaultSourceName(sourceDef)), - new EventTypeInfo()); - } - - // SourceFunction - if (eventSourceProvider instanceof FlinkSourceFunctionProvider) { - FlinkSourceFunctionProvider sourceFunctionProvider = - (FlinkSourceFunctionProvider) eventSourceProvider; - DataStreamSource stream = - env.addSource(sourceFunctionProvider.getSourceFunction(), new EventTypeInfo()); - sourceDef.getName().ifPresent(stream::name); - return stream; - } - - // Unknown provider type - throw new IllegalStateException( - String.format( - "Unsupported EventSourceProvider type \"%s\"", - eventSourceProvider.getClass().getCanonicalName())); - } - public DataSource createDataSource( SourceDef sourceDef, Configuration pipelineConfig, StreamExecutionEnvironment env) { // Search the data source factory diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/PartitioningTranslator.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/PartitioningTranslator.java index 58ee1a60972..11b2800b146 100644 --- a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/PartitioningTranslator.java +++ b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/PartitioningTranslator.java @@ -21,13 +21,13 @@ import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.function.HashFunctionProvider; -import org.apache.flink.cdc.runtime.partitioning.BatchRegularPrePartitionOperator; -import org.apache.flink.cdc.runtime.partitioning.DistributedPrePartitionOperator; +import org.apache.flink.cdc.runtime.partitioning.BatchRegularPrePartitionOperatorFactory; +import org.apache.flink.cdc.runtime.partitioning.DistributedPrePartitionOperatorFactory; import org.apache.flink.cdc.runtime.partitioning.EventPartitioner; import org.apache.flink.cdc.runtime.partitioning.PartitioningEvent; import org.apache.flink.cdc.runtime.partitioning.PartitioningEventKeySelector; import org.apache.flink.cdc.runtime.partitioning.PostPartitionProcessor; -import org.apache.flink.cdc.runtime.partitioning.RegularPrePartitionOperator; +import org.apache.flink.cdc.runtime.partitioning.RegularPrePartitionOperatorFactory; import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo; import org.apache.flink.cdc.runtime.typeutils.PartitioningEventTypeInfo; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -72,9 +72,9 @@ public DataStream translateRegular( isBatchMode ? "BatchPrePartition" : "PrePartition", new PartitioningEventTypeInfo(), isBatchMode - ? new BatchRegularPrePartitionOperator( + ? new BatchRegularPrePartitionOperatorFactory( downstreamParallelism, hashFunctionProvider) - : new RegularPrePartitionOperator( + : new RegularPrePartitionOperatorFactory( schemaOperatorID, downstreamParallelism, hashFunctionProvider)) @@ -97,7 +97,7 @@ public DataStream translateDistributed( return input.transform( "Partitioning", new PartitioningEventTypeInfo(), - new DistributedPrePartitionOperator( + new DistributedPrePartitionOperatorFactory( downstreamParallelism, hashFunctionProvider)) .setParallelism(upstreamParallelism) .partitionCustom(new EventPartitioner(), new PartitioningEventKeySelector()); diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/SchemaOperatorTranslator.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/SchemaOperatorTranslator.java index 8f31803b3f7..8bab253bd30 100644 --- a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/SchemaOperatorTranslator.java +++ b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/SchemaOperatorTranslator.java @@ -24,7 +24,7 @@ import org.apache.flink.cdc.common.sink.MetadataApplier; import org.apache.flink.cdc.common.utils.Preconditions; import org.apache.flink.cdc.composer.definition.RouteDef; -import org.apache.flink.cdc.runtime.operators.schema.regular.BatchSchemaOperator; +import org.apache.flink.cdc.runtime.operators.schema.regular.BatchSchemaOperatorFactory; import org.apache.flink.cdc.runtime.operators.schema.regular.SchemaOperator; import org.apache.flink.cdc.runtime.operators.schema.regular.SchemaOperatorFactory; import org.apache.flink.cdc.runtime.partitioning.PartitioningEvent; @@ -143,7 +143,7 @@ private DataStream addRegularSchemaBatchOperator( input.transform( "SchemaBatchOperator", new EventTypeInfo(), - new BatchSchemaOperator(routingRules, metadataApplier, timezone)); + new BatchSchemaOperatorFactory(routingRules, metadataApplier, timezone)); stream.uid(schemaOperatorUid).setParallelism(parallelism); return stream; } diff --git a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/TransformTranslator.java b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/TransformTranslator.java index a938d7ab05e..eed027a62d0 100644 --- a/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/TransformTranslator.java +++ b/flink-cdc-composer/src/main/java/org/apache/flink/cdc/composer/flink/translator/TransformTranslator.java @@ -56,10 +56,12 @@ public DataStream translatePreTransform( return input.transform( "Transform:Schema", new EventTypeInfo(), - generatePreTransform(transforms, udfFunctions, models, supportedMetadataColumns)); + generatePreTransformBuilder( + transforms, udfFunctions, models, supportedMetadataColumns) + .buildFactory()); } - private PreTransformOperator generatePreTransform( + private PreTransformOperatorBuilder generatePreTransformBuilder( List transforms, List udfFunctions, List models, @@ -86,7 +88,7 @@ private PreTransformOperator generatePreTransform( .addUdfFunctions( models.stream().map(this::modelToUDFTuple).collect(Collectors.toList())); - return preTransformFunctionBuilder.build(); + return preTransformFunctionBuilder; } public DataStream translatePostTransform( diff --git a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslatorTest.java b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslatorTest.java index 8ca8eb5cf4e..d0fa9843d66 100644 --- a/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslatorTest.java +++ b/flink-cdc-composer/src/test/java/org/apache/flink/cdc/composer/flink/translator/DataSinkTranslatorTest.java @@ -17,11 +17,11 @@ package org.apache.flink.cdc.composer.flink.translator; +import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.api.dag.Transformation; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.streaming.api.connector.sink2.WithPreWriteTopology; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -67,7 +67,7 @@ void testPreWriteWithoutCommitSink() { private static class EmptyEvent implements Event {} - private static class MockPreWriteWithoutCommitSink implements WithPreWriteTopology { + private static class MockPreWriteWithoutCommitSink implements Sink { private final String uid; @@ -75,7 +75,7 @@ public MockPreWriteWithoutCommitSink(String uid) { this.uid = uid; } - @Override + /** Discovered via reflection by DataSinkTranslator. */ public DataStream addPreWriteTopology(DataStream inputDataStream) { // return a new DataSteam with specified uid DataStream rebalance = inputDataStream.rebalance(); @@ -84,7 +84,7 @@ public DataStream addPreWriteTopology(DataStream inputDataStream) } @Override - public SinkWriter createWriter(InitContext context) throws IOException { + public SinkWriter createWriter(Sink.InitContext context) throws IOException { return null; } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/main/java/org/apache/flink/cdc/connectors/elasticsearch/sink/ElasticsearchDataSinkFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/main/java/org/apache/flink/cdc/connectors/elasticsearch/sink/ElasticsearchDataSinkFactory.java index 83f7c115218..6a142a705ad 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/main/java/org/apache/flink/cdc/connectors/elasticsearch/sink/ElasticsearchDataSinkFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/main/java/org/apache/flink/cdc/connectors/elasticsearch/sink/ElasticsearchDataSinkFactory.java @@ -20,13 +20,13 @@ import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.factories.DataSinkFactory; import org.apache.flink.cdc.common.factories.FactoryHelper; import org.apache.flink.cdc.common.pipeline.PipelineOptions; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.connectors.elasticsearch.config.ElasticsearchSinkOptions; import org.apache.flink.cdc.connectors.elasticsearch.v2.NetworkConfig; -import org.apache.flink.table.api.ValidationException; import org.apache.http.HttpHost; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/test/java/org/apache/flink/cdc/connectors/elasticsearch/sink/ElasticsearchDataSinkFactoryTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/test/java/org/apache/flink/cdc/connectors/elasticsearch/sink/ElasticsearchDataSinkFactoryTest.java index ef3691d312f..a5443d5c804 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/test/java/org/apache/flink/cdc/connectors/elasticsearch/sink/ElasticsearchDataSinkFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch/src/test/java/org/apache/flink/cdc/connectors/elasticsearch/sink/ElasticsearchDataSinkFactoryTest.java @@ -21,11 +21,11 @@ import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.factories.DataSinkFactory; import org.apache.flink.cdc.common.factories.FactoryHelper; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.composer.utils.FactoryDiscoveryUtils; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-fluss/src/main/java/org/apache/flink/cdc/connectors/fluss/sink/FlussMetaDataApplier.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-fluss/src/main/java/org/apache/flink/cdc/connectors/fluss/sink/FlussMetaDataApplier.java index e2cda4bd0ca..9119c472234 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-fluss/src/main/java/org/apache/flink/cdc/connectors/fluss/sink/FlussMetaDataApplier.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-fluss/src/main/java/org/apache/flink/cdc/connectors/fluss/sink/FlussMetaDataApplier.java @@ -24,9 +24,9 @@ import org.apache.flink.cdc.common.event.SchemaChangeEventType; import org.apache.flink.cdc.common.event.SchemaChangeEventTypeFamily; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.schema.Column; import org.apache.flink.cdc.common.sink.MetadataApplier; -import org.apache.flink.table.api.ValidationException; import org.apache.fluss.client.Connection; import org.apache.fluss.client.ConnectionFactory; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-fluss/src/test/java/org/apache/flink/cdc/connectors/fluss/factory/FlussDataSinkFactoryTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-fluss/src/test/java/org/apache/flink/cdc/connectors/fluss/factory/FlussDataSinkFactoryTest.java index 49ec01d3001..f84714ac7b9 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-fluss/src/test/java/org/apache/flink/cdc/connectors/fluss/factory/FlussDataSinkFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-fluss/src/test/java/org/apache/flink/cdc/connectors/fluss/factory/FlussDataSinkFactoryTest.java @@ -18,13 +18,13 @@ package org.apache.flink.cdc.connectors.fluss.factory; import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.factories.DataSinkFactory; import org.apache.flink.cdc.common.factories.FactoryHelper; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.composer.utils.FactoryDiscoveryUtils; import org.apache.flink.cdc.connectors.fluss.sink.FlussDataSink; import org.apache.flink.cdc.connectors.fluss.sink.FlussDataSinkOptions; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketAssignOperator.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketAssignOperator.java index ed0d61fd9f5..1ccb563e2e7 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketAssignOperator.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/BucketAssignOperator.java @@ -32,7 +32,6 @@ import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -93,7 +92,6 @@ public class BucketAssignOperator extends AbstractStreamOperator public BucketAssignOperator(Configuration conf, String schemaOperatorUid) { this.numBuckets = conf.getInteger(FlinkOptions.BUCKET_INDEX_NUM_BUCKETS); this.schemaOperatorUid = schemaOperatorUid; - this.chainingStrategy = ChainingStrategy.ALWAYS; } @Override diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/FlushEventAlignmentOperator.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/FlushEventAlignmentOperator.java index 615959916b8..c2972bcac5e 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/FlushEventAlignmentOperator.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/main/java/org/apache/flink/cdc/connectors/hudi/sink/bucket/FlushEventAlignmentOperator.java @@ -21,7 +21,6 @@ import org.apache.flink.cdc.common.event.FlushEvent; import org.apache.flink.cdc.runtime.operators.schema.regular.SchemaOperator; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -45,8 +44,7 @@ public class FlushEventAlignmentOperator extends AbstractStreamOperator, WithPreWriteTopology { +public class HudiSink implements Sink { private static final Logger LOG = LoggerFactory.getLogger(HudiSink.class); @@ -64,7 +63,10 @@ public SinkWriter createWriter(WriterInitContext context) throws IOExcept return DummySinkWriter.INSTANCE; } - @Override + /** + * Adds bucket-assign and flush-alignment operators (discovered via reflection by + * DataSinkTranslator). + */ public DataStream addPreWriteTopology(DataStream dataStream) { LOG.info("Building Hudi pre-write topology with bucket assignment and partitioning"); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/test/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkFactoryTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/test/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkFactoryTest.java index d6c1416265d..89d77209db6 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/test/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-hudi/src/test/java/org/apache/flink/cdc/connectors/hudi/sink/HudiDataSinkFactoryTest.java @@ -19,11 +19,11 @@ import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.factories.DataSinkFactory; import org.apache.flink.cdc.common.factories.FactoryHelper; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.composer.utils.FactoryDiscoveryUtils; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/main/java/org/apache/flink/cdc/connectors/iceberg/sink/v2/IcebergSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/main/java/org/apache/flink/cdc/connectors/iceberg/sink/v2/IcebergSink.java index db43adc3acc..ead1d862fad 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/main/java/org/apache/flink/cdc/connectors/iceberg/sink/v2/IcebergSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/main/java/org/apache/flink/cdc/connectors/iceberg/sink/v2/IcebergSink.java @@ -24,7 +24,6 @@ import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.api.connector.sink2.StatefulSinkWriter; import org.apache.flink.api.connector.sink2.SupportsWriterState; -import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink; import org.apache.flink.api.connector.sink2.WriterInitContext; import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.common.event.TableId; @@ -36,9 +35,6 @@ import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; import org.apache.flink.streaming.api.connector.sink2.CommittableMessageTypeInfo; import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; -import org.apache.flink.streaming.api.connector.sink2.WithPostCommitTopology; -import org.apache.flink.streaming.api.connector.sink2.WithPreCommitTopology; -import org.apache.flink.streaming.api.connector.sink2.WithPreWriteTopology; import org.apache.flink.streaming.api.datastream.DataStream; import java.time.ZoneId; @@ -48,13 +44,11 @@ import java.util.UUID; /** A {@link Sink} implementation for Apache Iceberg. */ -public class IcebergSink - implements Sink, - WithPreWriteTopology, - WithPreCommitTopology, - TwoPhaseCommittingSink, - WithPostCommitTopology, - SupportsWriterState { +/** + * Sink with addPreCommitTopology/addPostCommitTopology (discovered via reflection for Flink + * 1.x/2.x). + */ +public class IcebergSink implements Sink, SupportsWriterState { protected final Map catalogOptions; protected final Map tableOptions; @@ -81,24 +75,24 @@ public IcebergSink( this.operatorId = UUID.randomUUID().toString(); } - @Override + /** Adds compaction operator (discovered via reflection by DataSinkTranslator). */ public DataStream addPreWriteTopology(DataStream dataStream) { return dataStream; } - @Override + /** Used for two-phase commit (detected via reflection by DataSinkTranslator). */ public Committer createCommitter() { return new IcebergCommitter(catalogOptions); } - @Override + /** Overload with metrics (optional; used when available). */ public Committer createCommitter( CommitterInitContext committerInitContext) { SinkCommitterMetricGroup metricGroup = committerInitContext.metricGroup(); return new IcebergCommitter(catalogOptions, metricGroup); } - @Override + /** Used for two-phase commit (detected via reflection by DataSinkTranslator). */ public SimpleVersionedSerializer getCommittableSerializer() { return new WriteResultWrapperSerializer(); } @@ -161,7 +155,7 @@ public SimpleVersionedSerializer getWriterStateSerializer() return new IcebergWriterStateSerializer(); } - @Override + /** Discovered via reflection by DataSinkTranslator. */ public DataStream> addPreCommitTopology( DataStream> committables) { // Refer to @@ -169,12 +163,12 @@ public DataStream> addPreCommitTopology( return committables.global(); } - @Override + /** Used by pre-commit topology (from removed WithPreCommitTopology interface). */ public SimpleVersionedSerializer getWriteResultSerializer() { return new WriteResultWrapperSerializer(); } - @Override + /** Discovered via reflection by DataSinkTranslator. */ public void addPostCommitTopology( DataStream> committableMessageDataStream) { if (compactionOptions.isEnabled()) { diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/test/java/org/apache/flink/cdc/connectors/iceberg/sink/IcebergDataSinkFactoryTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/test/java/org/apache/flink/cdc/connectors/iceberg/sink/IcebergDataSinkFactoryTest.java index 10f89e16d08..1752fed141f 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/test/java/org/apache/flink/cdc/connectors/iceberg/sink/IcebergDataSinkFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-iceberg/src/test/java/org/apache/flink/cdc/connectors/iceberg/sink/IcebergDataSinkFactoryTest.java @@ -20,13 +20,13 @@ import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.event.CreateTableEvent; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.factories.DataSinkFactory; import org.apache.flink.cdc.common.factories.FactoryHelper; import org.apache.flink.cdc.common.schema.Schema; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.common.types.DataTypes; import org.apache.flink.cdc.composer.utils.FactoryDiscoveryUtils; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkFactoryTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkFactoryTest.java index f7da80dd23c..3342166c839 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-kafka/src/test/java/org/apache/flink/cdc/connectors/kafka/sink/KafkaDataSinkFactoryTest.java @@ -18,11 +18,11 @@ package org.apache.flink.cdc.connectors.kafka.sink; import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.factories.DataSinkFactory; import org.apache.flink.cdc.common.factories.FactoryHelper; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.composer.utils.FactoryDiscoveryUtils; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/main/java/org/apache/flink/cdc/connectors/maxcompute/coordinator/SessionManageOperator.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/main/java/org/apache/flink/cdc/connectors/maxcompute/coordinator/SessionManageOperator.java index ed9d3ecf2cd..044b7128b05 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/main/java/org/apache/flink/cdc/connectors/maxcompute/coordinator/SessionManageOperator.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/main/java/org/apache/flink/cdc/connectors/maxcompute/coordinator/SessionManageOperator.java @@ -48,7 +48,6 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -102,7 +101,6 @@ public class SessionManageOperator extends AbstractStreamOperator private transient boolean endOfInput; public SessionManageOperator(MaxComputeOptions options, OperatorID schemaOperatorUid) { - this.chainingStrategy = ChainingStrategy.ALWAYS; this.options = options; this.schemaOperatorUid = schemaOperatorUid; } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/main/java/org/apache/flink/cdc/connectors/maxcompute/sink/MaxComputeEventSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/main/java/org/apache/flink/cdc/connectors/maxcompute/sink/MaxComputeEventSink.java index f860a003979..47353d3bed5 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/main/java/org/apache/flink/cdc/connectors/maxcompute/sink/MaxComputeEventSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-maxcompute/src/main/java/org/apache/flink/cdc/connectors/maxcompute/sink/MaxComputeEventSink.java @@ -26,14 +26,13 @@ import org.apache.flink.cdc.connectors.maxcompute.options.MaxComputeOptions; import org.apache.flink.cdc.connectors.maxcompute.options.MaxComputeWriteOptions; import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo; -import org.apache.flink.streaming.api.connector.sink2.WithPreWriteTopology; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import java.io.IOException; /** A {@link Sink} of {@link Event} to MaxCompute. */ -public class MaxComputeEventSink implements Sink, WithPreWriteTopology { +public class MaxComputeEventSink implements Sink { private static final long serialVersionUID = 1L; private final MaxComputeOptions options; private final MaxComputeWriteOptions writeOptions; @@ -43,7 +42,7 @@ public MaxComputeEventSink(MaxComputeOptions options, MaxComputeWriteOptions wri this.writeOptions = writeOptions; } - @Override + /** Adds session-manage operator (discovered via reflection by DataSinkTranslator). */ public DataStream addPreWriteTopology(DataStream inputDataStream) { SingleOutputStreamOperator stream = inputDataStream.transform( diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/pom.xml index ec528cde751..e7bc6e62fae 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/pom.xml @@ -38,6 +38,14 @@ limitations under the License. ${project.version} + + + com.mysql + mysql-connector-j + 8.0.33 + provided + + org.apache.flink flink-connector-mysql-cdc diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java index 1b3540da0bb..0ab0543797a 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/main/java/org/apache/flink/cdc/connectors/mysql/factory/MySqlDataSourceFactory.java @@ -21,6 +21,7 @@ import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.factories.DataSourceFactory; import org.apache.flink.cdc.common.factories.Factory; import org.apache.flink.cdc.common.factories.FactoryHelper; @@ -39,7 +40,6 @@ import org.apache.flink.cdc.connectors.mysql.utils.MySqlSchemaUtils; import org.apache.flink.cdc.connectors.mysql.utils.OptionUtils; import org.apache.flink.cdc.debezium.table.DebeziumOptions; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.ObjectPath; import com.mysql.cj.conf.PropertyKey; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlDataSourceFactoryTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlDataSourceFactoryTest.java index 74f3ef52a3c..067b3a085c7 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlDataSourceFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-mysql/src/test/java/org/apache/flink/cdc/connectors/mysql/source/MySqlDataSourceFactoryTest.java @@ -19,10 +19,10 @@ import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.factories.Factory; import org.apache.flink.cdc.connectors.mysql.factory.MySqlDataSourceFactory; import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.ObjectPath; import org.junit.jupiter.api.Test; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/factory/OracleDataSourceFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/factory/OracleDataSourceFactory.java index a2b4647fc72..7771dcc1f62 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/factory/OracleDataSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-oracle/src/main/java/org/apache/flink/cdc/connectors/oracle/factory/OracleDataSourceFactory.java @@ -21,6 +21,7 @@ import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.factories.DataSourceFactory; import org.apache.flink.cdc.common.factories.Factory; import org.apache.flink.cdc.common.factories.FactoryHelper; @@ -35,7 +36,6 @@ import org.apache.flink.cdc.connectors.oracle.source.config.OracleSourceConfigFactory; import org.apache.flink.cdc.connectors.oracle.table.OracleReadableMetaData; import org.apache.flink.cdc.connectors.oracle.utils.OracleSchemaUtils; -import org.apache.flink.table.api.ValidationException; import java.time.Duration; import java.util.ArrayList; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/pom.xml index 274129b0c72..5d035b3b50e 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/pom.xml @@ -36,11 +36,12 @@ limitations under the License. - + org.apache.paimon paimon-flink-${flink.major.version} ${paimon.version} + provided @@ -257,34 +258,7 @@ limitations under the License. - - org.apache.maven.plugins - maven-shade-plugin - ${maven.shade.plugin.version} - - - shade-flink - package - - shade - - - false - - - org.apache.paimon:* - - - - - org.apache.kafka - shaded.paimon.org.apache.kafka - - - - - - + org.apache.maven.plugins diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonEventSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonEventSink.java index 9487a3cc22e..6dece51609e 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonEventSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonEventSink.java @@ -24,7 +24,6 @@ import org.apache.flink.cdc.connectors.paimon.sink.v2.bucket.BucketWrapperEventTypeInfo; import org.apache.flink.cdc.connectors.paimon.sink.v2.bucket.FlushEventAlignmentOperator; import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.streaming.api.connector.sink2.WithPreWriteTopology; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.paimon.flink.sink.MultiTableCommittable; @@ -35,7 +34,8 @@ import java.time.ZoneId; /** A {@link PaimonSink} to process {@link Event}. */ -public class PaimonEventSink extends PaimonSink implements WithPreWriteTopology { +/** Sink with addPreWriteTopology for schema/bucket operators (Flink 1.x/2.x compatible). */ +public class PaimonEventSink extends PaimonSink { public final String schemaOperatorUid; @@ -52,7 +52,10 @@ public PaimonEventSink( this.zoneId = zoneId; } - @Override + /** + * Adds bucket-assign and flush-alignment operators (discovered via reflection by + * DataSinkTranslator). + */ public DataStream addPreWriteTopology(DataStream dataStream) { // Shuffle by key hash => Assign bucket => Shuffle by bucket. return dataStream diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonSink.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonSink.java index ca249c20c9f..cb980253cd8 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonSink.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonSink.java @@ -28,7 +28,6 @@ import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; import org.apache.flink.streaming.api.connector.sink2.CommittableMessageTypeInfo; -import org.apache.flink.streaming.api.connector.sink2.WithPreCommitTopology; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.paimon.flink.sink.FlinkStreamPartitioner; @@ -43,9 +42,11 @@ /** * A {@link Sink} for Paimon. Maintain this package until Paimon has it own sinkV2 implementation. */ +/** + * Sink with createCommitter/getCommittableSerializer (detected via reflection for Flink 1.x/2.x). + */ public class PaimonSink - implements WithPreCommitTopology, - SupportsWriterState { + implements Sink, SupportsWriterState { // provided a default commit user. public static final String DEFAULT_COMMIT_USER = "admin"; @@ -70,7 +71,16 @@ public PaimonSink( } @Override - public PaimonWriter createWriter(InitContext context) { + public PaimonWriter createWriter(Sink.InitContext context) { + long lastCheckpointId = + context.getRestoredCheckpointId() + .orElse(CheckpointIDCounter.INITIAL_CHECKPOINT_ID - 1); + return new PaimonWriter<>( + catalogOptions, context.metricGroup(), commitUser, serializer, lastCheckpointId); + } + + @Override + public PaimonWriter createWriter(WriterInitContext context) { long lastCheckpointId = context.getRestoredCheckpointId() .orElse(CheckpointIDCounter.INITIAL_CHECKPOINT_ID - 1); @@ -94,18 +104,18 @@ public StatefulSinkWriter restoreWriter( lastCheckpointId); } - @Override + /** Used for two-phase commit (detected via reflection by DataSinkTranslator). */ public Committer createCommitter() { return new PaimonCommitter(catalogOptions, commitUser); } - @Override + /** Used for two-phase commit (detected via reflection by DataSinkTranslator). */ public SimpleVersionedSerializer getCommittableSerializer() { CommitMessageSerializer fileSerializer = new CommitMessageSerializer(); return new MultiTableCommittableSerializer(fileSerializer); } - @Override + /** Discovered via reflection by DataSinkTranslator. */ public DataStream> addPreCommitTopology( DataStream> committables) { TypeInformation> typeInformation = diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriter.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriter.java index 555086f41f9..9468464c81b 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriter.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PaimonWriter.java @@ -17,9 +17,9 @@ package org.apache.flink.cdc.connectors.paimon.sink.v2; +import org.apache.flink.api.connector.sink2.CommittingSinkWriter; import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.sink2.StatefulSinkWriter; -import org.apache.flink.api.connector.sink2.TwoPhaseCommittingSink; import org.apache.flink.cdc.common.event.DataChangeEvent; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.io.disk.iomanager.IOManager; @@ -53,7 +53,7 @@ /** A {@link Sink} to write {@link DataChangeEvent} to Paimon storage. */ public class PaimonWriter - implements TwoPhaseCommittingSink.PrecommittingSinkWriter, + implements CommittingSinkWriter, StatefulSinkWriter { private static final Logger LOG = LoggerFactory.getLogger(PaimonWriter.class); diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PreCommitOperator.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PreCommitOperator.java index 6bcca94ff05..8758bcbcbc0 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PreCommitOperator.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/PreCommitOperator.java @@ -86,8 +86,8 @@ public void initializeState(StateInitializationContext context) throws Exception true, context.isRestored(), context.getOperatorStateStore(), - getRuntimeContext().getNumberOfParallelSubtasks(), - getRuntimeContext().getIndexOfThisSubtask())); + getRuntimeContext().getTaskInfo().getNumberOfParallelSubtasks(), + getRuntimeContext().getTaskInfo().getIndexOfThisSubtask())); } } diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/bucket/BucketAssignOperator.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/bucket/BucketAssignOperator.java index faa832fc705..466dcd8d4d4 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/bucket/BucketAssignOperator.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/bucket/BucketAssignOperator.java @@ -39,7 +39,6 @@ import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -97,7 +96,6 @@ public class BucketAssignOperator extends AbstractStreamOperator public BucketAssignOperator( Options catalogOptions, String schemaOperatorUid, ZoneId zoneId, String commitUser) { this.catalogOptions = catalogOptions; - this.chainingStrategy = ChainingStrategy.ALWAYS; this.schemaOperatorUid = schemaOperatorUid; this.commitUser = commitUser; this.zoneId = zoneId; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/bucket/BucketWrapperEventTypeInfo.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/bucket/BucketWrapperEventTypeInfo.java index be1c54bfa06..2d234039170 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/bucket/BucketWrapperEventTypeInfo.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/bucket/BucketWrapperEventTypeInfo.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.connectors.paimon.sink.v2.bucket; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.cdc.common.event.Event; @@ -63,6 +64,12 @@ public TypeSerializer createSerializer(ExecutionConfig config) { return BucketWrapperEventSerializer.INSTANCE; } + /** For Flink 2.2+ (TypeInformation.createSerializer now takes SerializerConfig). */ + @Override + public TypeSerializer createSerializer(SerializerConfig config) { + return BucketWrapperEventSerializer.INSTANCE; + } + @Override public String toString() { return "BucketWrapperEvent"; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/bucket/FlushEventAlignmentOperator.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/bucket/FlushEventAlignmentOperator.java index 70f651e433b..ae166060147 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/bucket/FlushEventAlignmentOperator.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/main/java/org/apache/flink/cdc/connectors/paimon/sink/v2/bucket/FlushEventAlignmentOperator.java @@ -21,7 +21,6 @@ import org.apache.flink.cdc.common.event.FlushEvent; import org.apache.flink.cdc.runtime.operators.schema.regular.SchemaOperator; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -45,8 +44,7 @@ public class FlushEventAlignmentOperator extends AbstractStreamOperator private transient int currentSubTaskId; public FlushEventAlignmentOperator() { - // It's necessary to avoid unpredictable outcomes of Event shuffling. - this.chainingStrategy = ChainingStrategy.ALWAYS; + // Chaining is set via operator factory for Flink 2.x compatibility. } @Override diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonDataSinkFactoryTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonDataSinkFactoryTest.java index dcbbdd68241..5f3f929ca46 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonDataSinkFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-paimon/src/test/java/org/apache/flink/cdc/connectors/paimon/sink/PaimonDataSinkFactoryTest.java @@ -19,13 +19,13 @@ import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.factories.DataSinkFactory; import org.apache.flink.cdc.common.factories.FactoryHelper; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.common.sink.FlinkSinkProvider; import org.apache.flink.cdc.composer.utils.FactoryDiscoveryUtils; import org.apache.flink.cdc.connectors.paimon.sink.v2.PaimonEventSink; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/factory/PostgresDataSourceFactory.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/factory/PostgresDataSourceFactory.java index 578b6e0f8c6..569395ea177 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/factory/PostgresDataSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/main/java/org/apache/flink/cdc/connectors/postgres/factory/PostgresDataSourceFactory.java @@ -21,6 +21,7 @@ import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.Configuration; import org.apache.flink.cdc.common.event.TableId; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.factories.DataSourceFactory; import org.apache.flink.cdc.common.factories.Factory; import org.apache.flink.cdc.common.factories.FactoryHelper; @@ -34,7 +35,6 @@ import org.apache.flink.cdc.connectors.postgres.source.config.PostgresSourceConfigFactory; import org.apache.flink.cdc.connectors.postgres.table.PostgreSQLReadableMetadata; import org.apache.flink.cdc.connectors.postgres.utils.PostgresSchemaUtils; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.data.RowData; import org.slf4j.Logger; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/test/java/org/apache/flink/cdc/connectors/postgres/factory/PostgresDataSourceFactoryTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/test/java/org/apache/flink/cdc/connectors/postgres/factory/PostgresDataSourceFactoryTest.java index 473c2d61ced..22dc3dcbde0 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/test/java/org/apache/flink/cdc/connectors/postgres/factory/PostgresDataSourceFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-postgres/src/test/java/org/apache/flink/cdc/connectors/postgres/factory/PostgresDataSourceFactoryTest.java @@ -20,10 +20,10 @@ import org.apache.flink.cdc.common.annotation.Internal; import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.factories.Factory; import org.apache.flink.cdc.connectors.postgres.PostgresTestBase; import org.apache.flink.cdc.connectors.postgres.source.PostgresDataSource; -import org.apache.flink.table.api.ValidationException; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksDataSinkFactoryTest.java b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksDataSinkFactoryTest.java index 4e72782547b..f00b346d89c 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksDataSinkFactoryTest.java +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-starrocks/src/test/java/org/apache/flink/cdc/connectors/starrocks/sink/StarRocksDataSinkFactoryTest.java @@ -19,11 +19,11 @@ import org.apache.flink.cdc.common.configuration.ConfigOption; import org.apache.flink.cdc.common.configuration.Configuration; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.factories.DataSinkFactory; import org.apache.flink.cdc.common.factories.FactoryHelper; import org.apache.flink.cdc.common.sink.DataSink; import org.apache.flink.cdc.composer.utils.FactoryDiscoveryUtils; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap; diff --git a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/pom.xml b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/pom.xml index 3eda26dade8..b461d348a55 100644 --- a/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/pom.xml +++ b/flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values/pom.xml @@ -31,6 +31,11 @@ limitations under the License. + + org.apache.flink + flink-cdc-flink-compat-flink1 + ${project.version} + org.apache.flink flink-runtime diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/IncrementalSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/IncrementalSource.java index dfee377562f..e5566798636 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/IncrementalSource.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/IncrementalSource.java @@ -49,8 +49,6 @@ import org.apache.flink.cdc.connectors.base.source.utils.hooks.SnapshotPhaseHooks; import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; import org.apache.flink.connector.base.source.reader.RecordEmitter; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.flink.util.FlinkRuntimeException; @@ -115,8 +113,6 @@ public IncrementalSourceReader createReader(SourceReaderContext readerCont throws Exception { // create source config for the given subtask (e.g. unique server id) C sourceConfig = configFactory.create(readerContext.getIndexOfSubtask()); - FutureCompletingBlockingQueue> elementsQueue = - new FutureCompletingBlockingQueue<>(); final SourceReaderMetrics sourceReaderMetrics = new SourceReaderMetrics(readerContext.metricGroup()); @@ -132,7 +128,6 @@ public IncrementalSourceReader createReader(SourceReaderContext readerCont incrementalSourceReaderContext, snapshotHooks); return new IncrementalSourceReader<>( - elementsQueue, splitReaderSupplier, createRecordEmitter(sourceConfig, sourceReaderMetrics), readerContext.getConfiguration(), diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/meta/split/SourceSplitSerializer.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/meta/split/SourceSplitSerializer.java index 80e9d3aae97..b14a8b811a9 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/meta/split/SourceSplitSerializer.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/meta/split/SourceSplitSerializer.java @@ -35,6 +35,7 @@ import io.debezium.relational.history.TableChanges.TableChange; import java.io.IOException; +import java.lang.reflect.Method; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashMap; @@ -145,7 +146,7 @@ public SourceSplitBase deserializeSplit(int version, byte[] serialized) throws I } TableId tableId = TableId.parse(in.readUTF(), useCatalogBeforeSchema); String splitId = in.readUTF(); - RowType splitKeyType = (RowType) LogicalTypeParser.parse(in.readUTF()); + RowType splitKeyType = parseRowTypeCompat(in.readUTF()); Object[] splitBoundaryStart = SerializerUtils.serializedStringToRow(in.readUTF()); Object[] splitBoundaryEnd = SerializerUtils.serializedStringToRow(in.readUTF()); Offset highWatermark = readOffsetPosition(version, in); @@ -198,6 +199,31 @@ public SourceSplitBase deserializeSplit(int version, byte[] serialized) throws I } } + /** + * Parses a serialized {@link RowType} in a way that is compatible with both Flink 1.20 and + * Flink 2.2+. Flink 1.20 exposes LogicalTypeParser.parse(String) while Flink 2.2 moved to + * LogicalTypeParser.parse(String, ClassLoader). We use reflection to avoid a static dependency + * on either specific signature. + */ + private static RowType parseRowTypeCompat(String serialized) throws IOException { + try { + try { + // Flink ≤ 1.20: LogicalTypeParser.parse(String) + Method m = LogicalTypeParser.class.getMethod("parse", String.class); + return (RowType) m.invoke(null, serialized); + } catch (NoSuchMethodException ignore) { + // Flink ≥ 2.2: LogicalTypeParser.parse(String, ClassLoader) + Method m = + LogicalTypeParser.class.getMethod("parse", String.class, ClassLoader.class); + return (RowType) + m.invoke(null, serialized, Thread.currentThread().getContextClassLoader()); + } + } catch (Exception e) { + throw new IOException( + "Failed to parse RowType from serialized string: " + serialized, e); + } + } + public static void writeTableSchemas( Map tableSchemas, DataOutputSerializer out) throws IOException { FlinkJsonTableChangeSerializer jsonSerializer = new FlinkJsonTableChangeSerializer(); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceReader.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceReader.java index 01a2924a095..8360af4f00c 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceReader.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceReader.java @@ -44,10 +44,7 @@ import org.apache.flink.cdc.connectors.base.source.meta.split.StreamSplitState; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.RecordEmitter; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; -import org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager; -import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Preconditions; @@ -110,7 +107,6 @@ public class IncrementalSourceReader private final IncrementalSourceReaderContext incrementalSourceReaderContext; public IncrementalSourceReader( - FutureCompletingBlockingQueue> elementQueue, Supplier> splitReaderSupplier, RecordEmitter recordEmitter, Configuration config, @@ -119,8 +115,7 @@ public IncrementalSourceReader( SourceSplitSerializer sourceSplitSerializer, DataSourceDialect dialect) { super( - elementQueue, - new SingleThreadFetcherManager<>(elementQueue, splitReaderSupplier::get), + splitReaderSupplier::get, recordEmitter, config, incrementalSourceReaderContext.getSourceReaderContext()); diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceReaderWithCommit.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceReaderWithCommit.java index 2f7f5f1c26a..7a7d52987ad 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceReaderWithCommit.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/reader/IncrementalSourceReaderWithCommit.java @@ -26,7 +26,6 @@ import org.apache.flink.cdc.connectors.base.source.meta.split.StreamSplit; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.RecordEmitter; -import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,7 +46,6 @@ public class IncrementalSourceReaderWithCommit extends IncrementalSourceReader { private long maxCompletedCheckpointId; public IncrementalSourceReaderWithCommit( - FutureCompletingBlockingQueue elementQueue, Supplier supplier, RecordEmitter recordEmitter, Configuration config, @@ -56,7 +54,6 @@ public IncrementalSourceReaderWithCommit( SourceSplitSerializer sourceSplitSerializer, DataSourceDialect dialect) { super( - elementQueue, supplier, recordEmitter, config, diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/utils/JdbcChunkUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/utils/JdbcChunkUtils.java index b6de3d211c7..452836d35a7 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/utils/JdbcChunkUtils.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-cdc-base/src/main/java/org/apache/flink/cdc/connectors/base/source/utils/JdbcChunkUtils.java @@ -17,7 +17,7 @@ package org.apache.flink.cdc.connectors.base.source.utils; -import org.apache.flink.table.api.ValidationException; +import org.apache.flink.cdc.common.exceptions.ValidationException; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.Column; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/utils/Db2Utils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/utils/Db2Utils.java index 34e328dadca..d01dc0d18de 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/utils/Db2Utils.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/source/utils/Db2Utils.java @@ -17,9 +17,9 @@ package org.apache.flink.cdc.connectors.db2.source.utils; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset; import org.apache.flink.cdc.connectors.db2.source.offset.LsnOffset; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.FlinkRuntimeException; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSourceFactory.java index 529a8f0bc48..f86dcdc2a01 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-db2-cdc/src/main/java/org/apache/flink/cdc/connectors/db2/table/Db2TableSourceFactory.java @@ -17,13 +17,13 @@ package org.apache.flink.cdc.connectors.db2.table; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.connectors.base.options.StartupOptions; import org.apache.flink.cdc.connectors.db2.utils.OptionUtils; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.factories.DynamicTableSourceFactory; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableSourceFactory.java index 71770cf1e38..01ac01a581f 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mongodb-cdc/src/main/java/org/apache/flink/cdc/connectors/mongodb/table/MongoDBTableSourceFactory.java @@ -17,12 +17,12 @@ package org.apache.flink.cdc.connectors.mongodb.table; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.connectors.base.options.StartupOptions; import org.apache.flink.cdc.connectors.base.utils.OptionUtils; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/pom.xml index 5ce4f9a527b..d665e5714f1 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/pom.xml +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/pom.xml @@ -48,6 +48,24 @@ limitations under the License. io.debezium debezium-connector-mysql ${debezium.version} + + + com.mysql + mysql-connector-j + + + mysql + mysql-connector-java + + + + + + + com.mysql + mysql-connector-j + 8.0.33 + provided diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/MySqlValidator.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/MySqlValidator.java index 3f5f18befe2..ebf68594491 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/MySqlValidator.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/MySqlValidator.java @@ -17,12 +17,12 @@ package org.apache.flink.cdc.connectors.mysql; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfig; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions; import org.apache.flink.cdc.debezium.Validator; import org.apache.flink.table.api.TableException; -import org.apache.flink.table.api.ValidationException; import io.debezium.jdbc.JdbcConnection; import org.slf4j.Logger; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/utils/ChunkUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/utils/ChunkUtils.java index 794abd2b5f9..75df2fabdd9 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/utils/ChunkUtils.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/source/utils/ChunkUtils.java @@ -17,9 +17,9 @@ package org.apache.flink.cdc.connectors.mysql.source.utils; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils; import org.apache.flink.cdc.connectors.mysql.schema.MySqlTypeUtils; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Preconditions; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSourceFactory.java index 5ea430d94e7..57d4845a724 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/main/java/org/apache/flink/cdc/connectors/mysql/table/MySqlTableSourceFactory.java @@ -17,6 +17,7 @@ package org.apache.flink.cdc.connectors.mysql.table; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions; import org.apache.flink.cdc.connectors.mysql.source.config.ServerIdRange; import org.apache.flink.cdc.connectors.mysql.source.offset.BinlogOffset; @@ -27,7 +28,6 @@ import org.apache.flink.cdc.debezium.utils.JdbcUrlUtils; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.source.DynamicTableSource; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/MySqlValidatorTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/MySqlValidatorTest.java index 6abccb5c6fc..ec1b27fb008 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/MySqlValidatorTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/MySqlValidatorTest.java @@ -19,6 +19,7 @@ import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.connectors.mysql.source.MySqlSource; import org.apache.flink.cdc.connectors.mysql.source.assigners.state.PendingSplitsState; import org.apache.flink.cdc.connectors.mysql.source.split.MySqlSplit; @@ -26,7 +27,6 @@ import org.apache.flink.cdc.connectors.mysql.testutils.MySqlVersion; import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; import org.apache.flink.cdc.debezium.DebeziumSourceFunction; -import org.apache.flink.table.api.ValidationException; import org.apache.kafka.connect.source.SourceRecord; import org.assertj.core.api.Assertions; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlConnectorITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlConnectorITCase.java index 54bbd848197..e36ec06126b 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlConnectorITCase.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-mysql-cdc/src/test/java/org/apache/flink/cdc/connectors/mysql/table/MySqlConnectorITCase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils; import org.apache.flink.cdc.connectors.mysql.source.MySqlSourceTestBase; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceConfigFactory; @@ -31,7 +32,6 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableResult; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.bridge.java.StreamStatementSet; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.planner.factories.TestValuesTableFactory; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/OracleValidator.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/OracleValidator.java index 05c3bee1b4b..c7b84707fac 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/OracleValidator.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/OracleValidator.java @@ -17,10 +17,10 @@ package org.apache.flink.cdc.connectors.oracle; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.connectors.oracle.util.OracleJdbcUrlUtils; import org.apache.flink.cdc.debezium.Validator; import org.apache.flink.table.api.TableException; -import org.apache.flink.table.api.ValidationException; import java.sql.Connection; import java.sql.DatabaseMetaData; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSourceFactory.java index 9f345574939..ba9b8048aa8 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/table/OracleTableSourceFactory.java @@ -17,13 +17,13 @@ package org.apache.flink.cdc.connectors.oracle.table; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.connectors.base.options.StartupOptions; import org.apache.flink.cdc.connectors.base.utils.OptionUtils; import org.apache.flink.cdc.debezium.table.DebeziumOptions; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.factories.DynamicTableSourceFactory; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/util/ChunkUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/util/ChunkUtils.java index e3865da0355..4237517ce36 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/util/ChunkUtils.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/main/java/org/apache/flink/cdc/connectors/oracle/util/ChunkUtils.java @@ -17,8 +17,8 @@ package org.apache.flink.cdc.connectors.oracle.util; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.connectors.oracle.source.utils.OracleTypeUtils; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.types.logical.RowType; import io.debezium.relational.Column; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleSourceReaderTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleSourceReaderTest.java index 06069fd6185..a79d7ed3c65 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleSourceReaderTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oracle-cdc/src/test/java/org/apache/flink/cdc/connectors/oracle/source/reader/OracleSourceReaderTest.java @@ -44,8 +44,6 @@ import org.apache.flink.cdc.connectors.oracle.testutils.RecordsFormatter; import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; import org.apache.flink.connector.base.source.reader.RecordEmitter; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; import org.apache.flink.core.io.InputStatus; import org.apache.flink.metrics.groups.SourceReaderMetricGroup; @@ -223,8 +221,6 @@ private IncrementalSourceReader createReader( private IncrementalSourceReader createReader( OracleSourceConfig configuration, SourceReaderContext readerContext) { - final FutureCompletingBlockingQueue> elementsQueue = - new FutureCompletingBlockingQueue<>(); final SourceReaderMetricGroup sourceReaderMetricGroup = readerContext.metricGroup(); final SourceReaderMetrics sourceReaderMetrics = new SourceReaderMetrics(sourceReaderMetricGroup); @@ -247,7 +243,6 @@ private IncrementalSourceReader createReader( incrementalSourceReaderContext, SnapshotPhaseHooks.empty()); return new IncrementalSourceReader<>( - elementsQueue, splitReaderSupplier, recordEmitter, readerContext.getConfiguration(), diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/PostgresSourceBuilder.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/PostgresSourceBuilder.java index 0f550ca96b3..67c6f7a8807 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/PostgresSourceBuilder.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/PostgresSourceBuilder.java @@ -29,7 +29,6 @@ import org.apache.flink.cdc.connectors.base.source.assigner.state.PendingSplitsState; import org.apache.flink.cdc.connectors.base.source.assigner.state.StreamPendingSplitsState; import org.apache.flink.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; -import org.apache.flink.cdc.connectors.base.source.meta.split.SourceRecords; import org.apache.flink.cdc.connectors.base.source.meta.split.SourceSplitBase; import org.apache.flink.cdc.connectors.base.source.metrics.SourceReaderMetrics; import org.apache.flink.cdc.connectors.base.source.reader.IncrementalSourceReaderContext; @@ -40,8 +39,6 @@ import org.apache.flink.cdc.connectors.postgres.source.offset.PostgresOffsetFactory; import org.apache.flink.cdc.connectors.postgres.source.reader.PostgresSourceReader; import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.util.FlinkRuntimeException; import io.debezium.relational.TableId; @@ -418,8 +415,6 @@ public PostgresSourceReader createReader(SourceReaderContext readerContext) // create source config for the given subtask (e.g. unique server id) PostgresSourceConfig sourceConfig = (PostgresSourceConfig) configFactory.create(readerContext.getIndexOfSubtask()); - FutureCompletingBlockingQueue> elementsQueue = - new FutureCompletingBlockingQueue<>(); final SourceReaderMetrics sourceReaderMetrics = new SourceReaderMetrics(readerContext.metricGroup()); @@ -435,7 +430,6 @@ public PostgresSourceReader createReader(SourceReaderContext readerContext) incrementalSourceReaderContext, snapshotHooks); return new PostgresSourceReader( - elementsQueue, splitReaderSupplier, createRecordEmitter(sourceConfig, sourceReaderMetrics), readerContext.getConfiguration(), diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/reader/PostgresSourceReader.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/reader/PostgresSourceReader.java index 30f77ed2548..74a429aba20 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/reader/PostgresSourceReader.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/reader/PostgresSourceReader.java @@ -38,7 +38,6 @@ import org.apache.flink.cdc.connectors.postgres.source.events.OffsetCommitEvent; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.RecordEmitter; -import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,7 +65,6 @@ public class PostgresSourceReader extends IncrementalSourceReaderWithCommit { private final int lsnCommitCheckpointsDelay; public PostgresSourceReader( - FutureCompletingBlockingQueue elementQueue, Supplier supplier, RecordEmitter recordEmitter, Configuration config, @@ -75,7 +73,6 @@ public PostgresSourceReader( SourceSplitSerializer sourceSplitSerializer, DataSourceDialect dialect) { super( - elementQueue, supplier, recordEmitter, config, diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/utils/ChunkUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/utils/ChunkUtils.java index 822b583341a..ec6f4d4f594 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/utils/ChunkUtils.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/source/utils/ChunkUtils.java @@ -17,8 +17,8 @@ package org.apache.flink.cdc.connectors.postgres.source.utils; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.types.logical.RowType; import io.debezium.relational.Column; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableFactory.java index 876f04a83bf..8b3703d8a55 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-postgres-cdc/src/main/java/org/apache/flink/cdc/connectors/postgres/table/PostgreSQLTableFactory.java @@ -17,6 +17,7 @@ package org.apache.flink.cdc.connectors.postgres.table; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.connectors.base.options.JdbcSourceOptions; import org.apache.flink.cdc.connectors.base.options.StartupMode; import org.apache.flink.cdc.connectors.base.options.StartupOptions; @@ -24,7 +25,6 @@ import org.apache.flink.cdc.debezium.table.DebeziumChangelogMode; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.factories.DynamicTableFactory; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/SqlServerValidator.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/SqlServerValidator.java index e37310b6814..82f61cec5ba 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/SqlServerValidator.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/SqlServerValidator.java @@ -17,9 +17,9 @@ package org.apache.flink.cdc.connectors.sqlserver; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.debezium.Validator; import org.apache.flink.table.api.TableException; -import org.apache.flink.table.api.ValidationException; import com.microsoft.sqlserver.jdbc.SQLServerDriver; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/utils/SqlServerUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/utils/SqlServerUtils.java index deca1e7b3b7..c3ea7c1d8a9 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/utils/SqlServerUtils.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/source/utils/SqlServerUtils.java @@ -17,10 +17,10 @@ package org.apache.flink.cdc.connectors.sqlserver.source.utils; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.connectors.base.source.meta.offset.Offset; import org.apache.flink.cdc.connectors.base.utils.ObjectUtils; import org.apache.flink.cdc.connectors.sqlserver.source.offset.LsnOffset; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.FlinkRuntimeException; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactory.java index a841366b20c..f24522a1f37 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/main/java/org/apache/flink/cdc/connectors/sqlserver/table/SqlServerTableFactory.java @@ -17,13 +17,13 @@ package org.apache.flink.cdc.connectors.sqlserver.table; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.connectors.base.options.StartupOptions; import org.apache.flink.cdc.connectors.base.utils.OptionUtils; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.factories.DynamicTableSourceFactory; diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/reader/SqlserverSourceReaderTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/reader/SqlserverSourceReaderTest.java index c50559d19ed..4df9dff9816 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/reader/SqlserverSourceReaderTest.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-sqlserver-cdc/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/reader/SqlserverSourceReaderTest.java @@ -43,8 +43,6 @@ import org.apache.flink.cdc.connectors.sqlserver.testutils.RecordsFormatter; import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema; import org.apache.flink.connector.base.source.reader.RecordEmitter; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; import org.apache.flink.core.io.InputStatus; import org.apache.flink.table.api.DataTypes; @@ -201,10 +199,6 @@ public IncrementalSourceReaderWithCommit createReader( SnapshotPhaseHooks snapshotHooks, int limit) throws Exception { - // create source config for the given subtask (e.g. unique server id) - FutureCompletingBlockingQueue> elementsQueue = - new FutureCompletingBlockingQueue<>(); - IncrementalSourceReaderContext incrementalSourceReaderContext = new IncrementalSourceReaderContext(readerContext); @@ -226,7 +220,6 @@ public IncrementalSourceReaderWithCommit createReader( limit); return new IncrementalSourceReaderWithCommit( - elementsQueue, splitReaderSupplier, recordEmitter, readerContext.getConfiguration(), diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/src/main/java/org/apache/flink/cdc/connectors/tidb/table/TiDBTableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/src/main/java/org/apache/flink/cdc/connectors/tidb/table/TiDBTableSourceFactory.java index b38175b71b5..ab2de453e17 100644 --- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/src/main/java/org/apache/flink/cdc/connectors/tidb/table/TiDBTableSourceFactory.java +++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-tidb-cdc/src/main/java/org/apache/flink/cdc/connectors/tidb/table/TiDBTableSourceFactory.java @@ -17,11 +17,11 @@ package org.apache.flink.cdc.connectors.tidb.table; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.connectors.tidb.table.utils.OptionUtils; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.factories.DynamicTableSourceFactory; diff --git a/flink-cdc-dist/pom.xml b/flink-cdc-dist/pom.xml index 132bea2cda2..6aa32db33c2 100644 --- a/flink-cdc-dist/pom.xml +++ b/flink-cdc-dist/pom.xml @@ -29,6 +29,8 @@ limitations under the License. true + + flink-cdc-dist-${revision}-${flink.major.version} @@ -52,9 +54,15 @@ limitations under the License. flink-cdc-composer ${project.version} + + org.apache.flink + ${flink.cdc.compat.artifact} + ${project.version} + + ${dist.jar.basename} org.apache.maven.plugins @@ -81,7 +89,7 @@ limitations under the License. false false false - ${project.artifactId}-${revision} + ${dist.jar.basename} * @@ -111,7 +119,7 @@ limitations under the License. gnu false false - flink-cdc-${project.version}-bin + flink-cdc-${project.version}-${flink.major.version}-bin ${basedir}/src/main/assembly/assembly.xml diff --git a/flink-cdc-dist/src/main/assembly/assembly.xml b/flink-cdc-dist/src/main/assembly/assembly.xml index c6d33eeea6d..68bf002f93e 100644 --- a/flink-cdc-dist/src/main/assembly/assembly.xml +++ b/flink-cdc-dist/src/main/assembly/assembly.xml @@ -25,11 +25,11 @@ limitations under the License. flink-cdc-${revision} - + - target/flink-cdc-dist-${revision}.jar + target/${project.build.finalName}.jar lib - flink-cdc-dist-${revision}.jar + ${project.build.finalName}.jar 0644 @@ -48,6 +48,17 @@ limitations under the License. + + + + lib + + org.apache.flink:flink-cdc-flink-compat-flink2 + + false + + + diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MaxComputeE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MaxComputeE2eITCase.java index 59d966bc2af..24c72bd2ff1 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MaxComputeE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MaxComputeE2eITCase.java @@ -19,13 +19,13 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.test.utils.TestUtils; import org.apache.flink.cdc.connectors.maxcompute.options.MaxComputeOptions; import org.apache.flink.cdc.connectors.maxcompute.utils.MaxComputeUtils; import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.runtime.client.JobStatusMessage; -import org.apache.flink.table.api.ValidationException; import com.aliyun.odps.Instance; import com.aliyun.odps.Odps; diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToHudiE2eITCase.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToHudiE2eITCase.java index 76936b6987b..3de85c4d178 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToHudiE2eITCase.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/MySqlToHudiE2eITCase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.test.utils.TestUtils; import org.apache.flink.cdc.connectors.mysql.testutils.UniqueDatabase; import org.apache.flink.cdc.pipeline.tests.utils.PipelineTestEnvironment; @@ -27,7 +28,6 @@ import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.runtime.client.JobStatusMessage; -import org.apache.flink.table.api.ValidationException; import org.apache.hudi.common.model.HoodieTableType; import org.assertj.core.api.Assertions; diff --git a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java index 641180580c7..fa3ecec7d20 100644 --- a/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java +++ b/flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests/src/test/java/org/apache/flink/cdc/pipeline/tests/utils/PipelineTestEnvironment.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.test.utils.TestUtils; import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer; import org.apache.flink.cdc.connectors.mysql.testutils.MySqlVersion; @@ -28,7 +29,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestOptions; import org.apache.flink.runtime.client.JobStatusMessage; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.util.TestLogger; import com.github.dockerjava.api.DockerClient; diff --git a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/org/apache/flink/cdc/connectors/tests/utils/FlinkContainerTestEnvironment.java b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/org/apache/flink/cdc/connectors/tests/utils/FlinkContainerTestEnvironment.java index 96173989c38..0d1ae70f674 100644 --- a/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/org/apache/flink/cdc/connectors/tests/utils/FlinkContainerTestEnvironment.java +++ b/flink-cdc-e2e-tests/flink-cdc-source-e2e-tests/src/test/java/org/apache/flink/cdc/connectors/tests/utils/FlinkContainerTestEnvironment.java @@ -19,6 +19,7 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.cdc.common.exceptions.ValidationException; import org.apache.flink.cdc.common.test.utils.TestUtils; import org.apache.flink.cdc.connectors.mysql.testutils.MySqlContainer; import org.apache.flink.cdc.connectors.mysql.testutils.MySqlVersion; @@ -28,7 +29,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.RestOptions; import org.apache.flink.runtime.client.JobStatusMessage; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.util.TestLogger; import com.fasterxml.jackson.core.Version; diff --git a/flink-cdc-flink-compat/flink-cdc-flink-compat-api/pom.xml b/flink-cdc-flink-compat/flink-cdc-flink-compat-api/pom.xml new file mode 100644 index 00000000000..5bc997d783f --- /dev/null +++ b/flink-cdc-flink-compat/flink-cdc-flink-compat-api/pom.xml @@ -0,0 +1,50 @@ + + + + + flink-cdc-flink-compat + org.apache.flink + ${revision} + + 4.0.0 + + flink-cdc-flink-compat-api + API for Flink version-specific source/sink binding (Flink 1.x vs 2.x). + + + + org.apache.flink + flink-cdc-common + ${project.version} + + + org.apache.flink + flink-streaming-java + ${flink.version} + provided + + + org.apache.flink + flink-core + ${flink.version} + provided + + + diff --git a/flink-cdc-flink-compat/flink-cdc-flink-compat-api/src/main/java/org/apache/flink/cdc/flink/compat/FlinkPipelineBridge.java b/flink-cdc-flink-compat/flink-cdc-flink-compat-api/src/main/java/org/apache/flink/cdc/flink/compat/FlinkPipelineBridge.java new file mode 100644 index 00000000000..5595491df34 --- /dev/null +++ b/flink-cdc-flink-compat/flink-cdc-flink-compat-api/src/main/java/org/apache/flink/cdc/flink/compat/FlinkPipelineBridge.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.flink.compat; + +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.sink.EventSinkProvider; +import org.apache.flink.cdc.common.source.EventSourceProvider; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import java.util.function.Function; + +/** + * Version-specific bridge for binding {@link EventSourceProvider} / {@link EventSinkProvider} to + * Flink DataStream API. + * + *

Flink 1.x implementation supports both Source/Sink V2 and legacy SourceFunction/SinkFunction. + * Flink 2.x implementation supports only Source/Sink V2. + * + *

Implementations are loaded via {@link java.util.ServiceLoader}; exactly one compat JAR + * (flink-cdc-flink-compat-flink1 or flink-cdc-flink-compat-flink2) must be on the classpath. + */ +@Internal +public interface FlinkPipelineBridge { + + /** + * Creates a {@link DataStreamSource} from the given {@link EventSourceProvider}. + * + * @param env execution environment + * @param eventSourceProvider provider (e.g. FlinkSourceProvider or FlinkSourceFunctionProvider + * on 1.x) + * @param sourceName name for the source operator + * @return the data stream source + * @throws IllegalStateException if the provider type is not supported by this bridge + */ + DataStreamSource createDataStreamSource( + StreamExecutionEnvironment env, + EventSourceProvider eventSourceProvider, + String sourceName); + + /** + * Adds the given {@link EventSinkProvider} as a sink to the input stream. + * + * @param input input event stream + * @param eventSinkProvider provider (e.g. FlinkSinkProvider or FlinkSinkFunctionProvider on + * 1.x) + * @param sinkName name for the sink operator + * @param isBatchMode whether the job is in batch mode + * @param schemaOperatorID schema operator ID for coordination + * @param uidGenerator generates operator UID from a suffix (e.g. "sink-writer" -> + * "prefix-sink-writer") + */ + void sinkTo( + DataStream input, + EventSinkProvider eventSinkProvider, + String sinkName, + boolean isBatchMode, + OperatorID schemaOperatorID, + Function uidGenerator); +} diff --git a/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/pom.xml b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/pom.xml new file mode 100644 index 00000000000..3fc361a7380 --- /dev/null +++ b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/pom.xml @@ -0,0 +1,60 @@ + + + + + flink-cdc-flink-compat + org.apache.flink + ${revision} + + 4.0.0 + + flink-cdc-flink-compat-flink1 + Flink 1.x compatibility: SourceFunction/SinkFunction providers and operators. + + + + org.apache.flink + flink-cdc-flink-compat-api + ${project.version} + + + org.apache.flink + flink-cdc-common + ${project.version} + + + org.apache.flink + flink-cdc-runtime + ${project.version} + + + org.apache.flink + flink-streaming-java + ${flink.version} + provided + + + org.apache.flink + flink-core + ${flink.version} + provided + + + diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/FlinkSinkFunctionProvider.java b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/java/org/apache/flink/cdc/common/sink/FlinkSinkFunctionProvider.java similarity index 92% rename from flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/FlinkSinkFunctionProvider.java rename to flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/java/org/apache/flink/cdc/common/sink/FlinkSinkFunctionProvider.java index 5e907bef6cb..56e3ebf451d 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/sink/FlinkSinkFunctionProvider.java +++ b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/java/org/apache/flink/cdc/common/sink/FlinkSinkFunctionProvider.java @@ -23,7 +23,8 @@ /** * {@code FlinkSinkFunctionProvider} is used to provide a Flink {@link SinkFunction} for writing - * events to external systems. + * events to external systems. This interface lives in the Flink 1.x compat module because + * SinkFunction was removed in Flink 2.x. */ @PublicEvolving public interface FlinkSinkFunctionProvider extends EventSinkProvider { diff --git a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/source/FlinkSourceFunctionProvider.java b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/java/org/apache/flink/cdc/common/source/FlinkSourceFunctionProvider.java similarity index 92% rename from flink-cdc-common/src/main/java/org/apache/flink/cdc/common/source/FlinkSourceFunctionProvider.java rename to flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/java/org/apache/flink/cdc/common/source/FlinkSourceFunctionProvider.java index 5713aac21a9..d08f865c3f5 100644 --- a/flink-cdc-common/src/main/java/org/apache/flink/cdc/common/source/FlinkSourceFunctionProvider.java +++ b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/java/org/apache/flink/cdc/common/source/FlinkSourceFunctionProvider.java @@ -23,7 +23,8 @@ /** * {@code FlinkSourceFunctionProvider} is used to provide a Flink {@link SourceFunction} for reading - * events from external systems. + * events from external systems. This interface lives in the Flink 1.x compat module because + * SourceFunction was removed in Flink 2.x. */ @PublicEvolving public interface FlinkSourceFunctionProvider extends EventSourceProvider { diff --git a/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/java/org/apache/flink/cdc/flink/compat/flink1/Flink1PipelineBridge.java b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/java/org/apache/flink/cdc/flink/compat/flink1/Flink1PipelineBridge.java new file mode 100644 index 00000000000..2d5d17f0370 --- /dev/null +++ b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/java/org/apache/flink/cdc/flink/compat/flink1/Flink1PipelineBridge.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.flink.compat.flink1; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.sink.EventSinkProvider; +import org.apache.flink.cdc.common.sink.FlinkSinkFunctionProvider; +import org.apache.flink.cdc.common.source.EventSourceProvider; +import org.apache.flink.cdc.common.source.FlinkSourceFunctionProvider; +import org.apache.flink.cdc.common.source.FlinkSourceProvider; +import org.apache.flink.cdc.flink.compat.FlinkPipelineBridge; +import org.apache.flink.cdc.runtime.operators.sink.BatchDataSinkFunctionOperator; +import org.apache.flink.cdc.runtime.operators.sink.DataSinkFunctionOperator; +import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.operators.StreamSink; +import org.apache.flink.streaming.api.transformations.LegacySinkTransformation; +import org.apache.flink.streaming.api.transformations.PhysicalTransformation; + +import java.util.function.Function; + +/** Flink 1.x implementation of {@link FlinkPipelineBridge}. */ +@Internal +public class Flink1PipelineBridge implements FlinkPipelineBridge { + + private static final String SINK_WRITER_PREFIX = "Sink Writer: "; + + @Override + public DataStreamSource createDataStreamSource( + StreamExecutionEnvironment env, + EventSourceProvider eventSourceProvider, + String sourceName) { + if (eventSourceProvider instanceof FlinkSourceProvider) { + FlinkSourceProvider sourceProvider = (FlinkSourceProvider) eventSourceProvider; + return env.fromSource( + sourceProvider.getSource(), + WatermarkStrategy.noWatermarks(), + sourceName, + new EventTypeInfo()); + } + if (eventSourceProvider instanceof FlinkSourceFunctionProvider) { + FlinkSourceFunctionProvider sourceFunctionProvider = + (FlinkSourceFunctionProvider) eventSourceProvider; + DataStreamSource stream = + env.addSource(sourceFunctionProvider.getSourceFunction(), new EventTypeInfo()); + stream.name(sourceName); + return stream; + } + throw new IllegalStateException( + String.format( + "Unsupported EventSourceProvider type \"%s\"", + eventSourceProvider.getClass().getCanonicalName())); + } + + @Override + public void sinkTo( + DataStream input, + EventSinkProvider eventSinkProvider, + String sinkName, + boolean isBatchMode, + OperatorID schemaOperatorID, + Function uidGenerator) { + if (!(eventSinkProvider instanceof FlinkSinkFunctionProvider)) { + throw new IllegalStateException( + String.format( + "Unsupported EventSinkProvider type \"%s\" for SinkFunction path", + eventSinkProvider.getClass().getCanonicalName())); + } + FlinkSinkFunctionProvider sinkFunctionProvider = + (FlinkSinkFunctionProvider) eventSinkProvider; + SinkFunction sinkFunction = sinkFunctionProvider.getSinkFunction(); + + StreamSink sinkOperator; + if (isBatchMode) { + sinkOperator = new BatchDataSinkFunctionOperator(sinkFunction); + } else { + sinkOperator = new DataSinkFunctionOperator(sinkFunction, schemaOperatorID); + } + StreamExecutionEnvironment executionEnvironment = input.getExecutionEnvironment(); + PhysicalTransformation transformation = + new LegacySinkTransformation<>( + input.getTransformation(), + SINK_WRITER_PREFIX + sinkName, + sinkOperator, + executionEnvironment.getParallelism(), + false); + String uid = (uidGenerator != null) ? uidGenerator.apply("sink-writer") : null; + if (uid != null) { + transformation.setUid(uid); + } + executionEnvironment.addOperator(transformation); + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkFunctionOperator.java b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkFunctionOperator.java similarity index 82% rename from flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkFunctionOperator.java rename to flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkFunctionOperator.java index 560909006d3..1e5c3109803 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkFunctionOperator.java +++ b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkFunctionOperator.java @@ -21,23 +21,18 @@ import org.apache.flink.cdc.common.event.Event; import org.apache.flink.cdc.runtime.operators.sink.exception.SinkWrapperException; import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** - * An operator that processes records to be written into a {@link SinkFunction} in batch mode. - * - *

The operator is a proxy of {@link StreamSink} in Flink. - * - *

The operator is always part of a sink pipeline and is the first operator. + * Batch-mode operator that writes into a {@link SinkFunction}. Lives in the Flink 1.x compat module + * because SinkFunction was removed in Flink 2.x. */ @Internal public class BatchDataSinkFunctionOperator extends StreamSink { public BatchDataSinkFunctionOperator(SinkFunction userFunction) { super(userFunction); - this.chainingStrategy = ChainingStrategy.ALWAYS; } @Override diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkFunctionOperator.java b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkFunctionOperator.java similarity index 83% rename from flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkFunctionOperator.java rename to flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkFunctionOperator.java index df0e7802b27..76261373947 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkFunctionOperator.java +++ b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkFunctionOperator.java @@ -30,7 +30,6 @@ import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.streaming.api.graph.StreamConfig; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -41,13 +40,8 @@ import java.util.Set; /** - * An operator that processes records to be written into a {@link - * org.apache.flink.streaming.api.functions.sink.SinkFunction}. - * - *

The operator is a proxy of {@link org.apache.flink.streaming.api.operators.StreamSink} in - * Flink. - * - *

The operator is always part of a sink pipeline and is the first operator. + * An operator that processes records to be written into a {@link SinkFunction}. Lives in the Flink + * 1.x compat module because SinkFunction was removed in Flink 2.x. */ @Internal public class DataSinkFunctionOperator extends StreamSink { @@ -55,14 +49,12 @@ public class DataSinkFunctionOperator extends StreamSink { private SchemaEvolutionClient schemaEvolutionClient; private final OperatorID schemaOperatorID; - /** A set of {@link TableId} that already processed {@link CreateTableEvent}. */ private final Set processedTableIds; public DataSinkFunctionOperator(SinkFunction userFunction, OperatorID schemaOperatorID) { super(userFunction); this.schemaOperatorID = schemaOperatorID; processedTableIds = new HashSet<>(); - this.chainingStrategy = ChainingStrategy.ALWAYS; } @Override @@ -88,23 +80,17 @@ public void processElement(StreamRecord element) throws Exception { Event event = element.getValue(); try { - // FlushEvent triggers flush if (event instanceof FlushEvent) { handleFlushEvent(((FlushEvent) event)); return; } - // CreateTableEvent marks the table as processed directly if (event instanceof CreateTableEvent) { processedTableIds.add(((CreateTableEvent) event).tableId()); super.processElement(element); return; } - // Check if the table is processed before emitting all other events, because we have to - // make - // sure that sink have a view of the full schema before processing any change events, - // including schema changes. ChangeEvent changeEvent = (ChangeEvent) event; if (!processedTableIds.contains(changeEvent.tableId())) { emitLatestSchema(changeEvent.tableId()); @@ -117,7 +103,6 @@ public void processElement(StreamRecord element) throws Exception { } } - // ----------------------------- Helper functions ------------------------------- private void handleFlushEvent(FlushEvent event) throws Exception { userFunction.finish(); if (event.getSchemaChangeEventType() != SchemaChangeEventType.CREATE_TABLE) { @@ -141,8 +126,6 @@ private void handleFlushEvent(FlushEvent event) throws Exception { private void emitLatestSchema(TableId tableId) throws Exception { Optional schema = schemaEvolutionClient.getLatestEvolvedSchema(tableId); if (schema.isPresent()) { - // request and process CreateTableEvent because SinkFunction need to retrieve - // Schema to deserialize RecordData after resuming job. super.processElement(new StreamRecord<>(new CreateTableEvent(tableId, schema.get()))); processedTableIds.add(tableId); } else { diff --git a/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/resources/META-INF/services/org.apache.flink.cdc.flink.compat.FlinkPipelineBridge b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/resources/META-INF/services/org.apache.flink.cdc.flink.compat.FlinkPipelineBridge new file mode 100644 index 00000000000..7b72cdc6ae1 --- /dev/null +++ b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink1/src/main/resources/META-INF/services/org.apache.flink.cdc.flink.compat.FlinkPipelineBridge @@ -0,0 +1 @@ +org.apache.flink.cdc.flink.compat.flink1.Flink1PipelineBridge diff --git a/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/pom.xml b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/pom.xml new file mode 100644 index 00000000000..a79c78c70aa --- /dev/null +++ b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/pom.xml @@ -0,0 +1,60 @@ + + + + + flink-cdc-flink-compat + org.apache.flink + ${revision} + + 4.0.0 + + flink-cdc-flink-compat-flink2 + Flink 2.x compatibility: Source/Sink V2 only (no SourceFunction/SinkFunction). + + + + org.apache.flink + flink-cdc-flink-compat-api + ${project.version} + + + org.apache.flink + flink-cdc-common + ${project.version} + + + org.apache.flink + flink-cdc-runtime + ${project.version} + + + org.apache.flink + flink-streaming-java + ${flink.version} + provided + + + org.apache.flink + flink-core + ${flink.version} + provided + + + diff --git a/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/src/main/java/org/apache/flink/api/connector/sink2/Sink$InitContext.java b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/src/main/java/org/apache/flink/api/connector/sink2/Sink$InitContext.java new file mode 100644 index 00000000000..b5060b995cf --- /dev/null +++ b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/src/main/java/org/apache/flink/api/connector/sink2/Sink$InitContext.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.connector.sink2; + +/** + * Compatibility stub for {@code Sink.InitContext} which was removed in Flink 2.x. Sink connector + * classes compiled against Flink 1.20 still reference {@code Sink$InitContext} in their bytecode + * (via {@code createWriter(Sink.InitContext)} method signatures). Java serialization introspects + * all method parameter types via {@code getDeclaredMethods0()}, causing {@code + * NoClassDefFoundError} on Flink 2.x if this class is absent. This empty stub makes the class + * loadable without providing any actual implementation. + */ +// CHECKSTYLE.OFF: TypeName +public interface Sink$InitContext extends InitContext {} +// CHECKSTYLE.ON: TypeName diff --git a/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/src/main/java/org/apache/flink/cdc/flink/compat/flink2/Flink2PipelineBridge.java b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/src/main/java/org/apache/flink/cdc/flink/compat/flink2/Flink2PipelineBridge.java new file mode 100644 index 00000000000..840eafaebb5 --- /dev/null +++ b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/src/main/java/org/apache/flink/cdc/flink/compat/flink2/Flink2PipelineBridge.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.flink.compat.flink2; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.sink.EventSinkProvider; +import org.apache.flink.cdc.common.source.EventSourceProvider; +import org.apache.flink.cdc.common.source.FlinkSourceProvider; +import org.apache.flink.cdc.flink.compat.FlinkPipelineBridge; +import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +import java.util.function.Function; + +/** + * Flink 2.x implementation of {@link FlinkPipelineBridge}. + * + *

Supports only Source/Sink V2 ({@link FlinkSourceProvider}, {@link FlinkSinkProvider}). + * SourceFunction and SinkFunction were removed in Flink 2.x; use compat-flink1 for Flink 1.x. + */ +@Internal +public class Flink2PipelineBridge implements FlinkPipelineBridge { + + private static final String UNSUPPORTED_SOURCE_MSG = + "Flink 2.x supports only FlinkSourceProvider (Source V2). " + + "SourceFunction was removed in Flink 2.x. Use flink-cdc-flink-compat-flink1 for Flink 1.x."; + private static final String UNSUPPORTED_SINK_MSG = + "Flink 2.x supports only FlinkSinkProvider (Sink V2). " + + "SinkFunction was removed in Flink 2.x. Use flink-cdc-flink-compat-flink1 for Flink 1.x."; + + @Override + public DataStreamSource createDataStreamSource( + StreamExecutionEnvironment env, + EventSourceProvider eventSourceProvider, + String sourceName) { + if (eventSourceProvider instanceof FlinkSourceProvider) { + FlinkSourceProvider sourceProvider = (FlinkSourceProvider) eventSourceProvider; + return env.fromSource( + sourceProvider.getSource(), + WatermarkStrategy.noWatermarks(), + sourceName, + new EventTypeInfo()); + } + throw new UnsupportedOperationException( + UNSUPPORTED_SOURCE_MSG + + " Got: " + + (eventSourceProvider == null + ? "null" + : eventSourceProvider.getClass().getName())); + } + + @Override + public void sinkTo( + DataStream input, + EventSinkProvider eventSinkProvider, + String sinkName, + boolean isBatchMode, + OperatorID schemaOperatorID, + Function uidGenerator) { + // In Flink 2.x only FlinkSinkProvider (Sink V2) is supported; the translator handles it + // directly. This path is only hit for legacy SinkFunction-based providers, which are + // unsupported in 2.x. + throw new UnsupportedOperationException( + UNSUPPORTED_SINK_MSG + + " Got: " + + (eventSinkProvider == null + ? "null" + : eventSinkProvider.getClass().getName())); + } +} diff --git a/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/src/main/java/org/apache/flink/table/catalog/Catalog.java b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/src/main/java/org/apache/flink/table/catalog/Catalog.java new file mode 100644 index 00000000000..4d1185f3fd4 --- /dev/null +++ b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/src/main/java/org/apache/flink/table/catalog/Catalog.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog; + +/** + * Minimal compatibility stub for Flink's Catalog from older table APIs. It exists so that + * connectors compiled against Flink 1.x table modules (for example certain Paimon artifacts) can + * still be loaded on Flink 2.2 runtimes where this type was removed or relocated. + */ +// CHECKSTYLE.OFF: TypeName +public interface Catalog {} +// CHECKSTYLE.ON: TypeName diff --git a/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/src/main/java/org/apache/flink/table/factories/CatalogFactory.java b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/src/main/java/org/apache/flink/table/factories/CatalogFactory.java new file mode 100644 index 00000000000..128e4d85269 --- /dev/null +++ b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/src/main/java/org/apache/flink/table/factories/CatalogFactory.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.factories; + +/** + * Minimal compatibility stub for Flink's CatalogFactory which existed in older Flink table APIs and + * is still referenced by some connector artifacts (for example Paimon compiled against Flink 1.x). + * This interface is intentionally empty and only restores the missing type so that such connectors + * can be loaded on Flink 2.2 runtimes. + */ +// CHECKSTYLE.OFF: TypeName +public interface CatalogFactory {} +// CHECKSTYLE.ON: TypeName diff --git a/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/src/main/resources/META-INF/services/org.apache.flink.cdc.flink.compat.FlinkPipelineBridge b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/src/main/resources/META-INF/services/org.apache.flink.cdc.flink.compat.FlinkPipelineBridge new file mode 100644 index 00000000000..71b7f1083c0 --- /dev/null +++ b/flink-cdc-flink-compat/flink-cdc-flink-compat-flink2/src/main/resources/META-INF/services/org.apache.flink.cdc.flink.compat.FlinkPipelineBridge @@ -0,0 +1 @@ +org.apache.flink.cdc.flink.compat.flink2.Flink2PipelineBridge diff --git a/flink-cdc-flink-compat/pom.xml b/flink-cdc-flink-compat/pom.xml new file mode 100644 index 00000000000..220fbc74ce1 --- /dev/null +++ b/flink-cdc-flink-compat/pom.xml @@ -0,0 +1,37 @@ + + + + + flink-cdc-parent + org.apache.flink + ${revision} + + 4.0.0 + + flink-cdc-flink-compat + pom + Flink version compatibility layer: Flink 1.x (SourceFunction/SinkFunction) vs Flink 2.x (Source/Sink V2). + + + flink-cdc-flink-compat-api + flink-cdc-flink-compat-flink1 + flink-cdc-flink-compat-flink2 + + diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/distributed/SchemaOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/distributed/SchemaOperator.java index 15c0571ba16..45f91afaf98 100755 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/distributed/SchemaOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/distributed/SchemaOperator.java @@ -97,7 +97,7 @@ public SchemaOperator( @Override public void open() throws Exception { super.open(); - subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); upstreamSchemaTable = HashBasedTable.create(); evolvedSchemaMap = new HashMap<>(); tableIdRouter = new TableIdRouter(routingRules); diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/BatchSchemaOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/BatchSchemaOperator.java index df990425afc..dacd8aa7626 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/BatchSchemaOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/BatchSchemaOperator.java @@ -32,7 +32,6 @@ import org.apache.flink.cdc.runtime.operators.schema.common.SchemaManager; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -70,7 +69,6 @@ public class BatchSchemaOperator extends AbstractStreamOperator public BatchSchemaOperator( List routingRules, MetadataApplier metadataApplier, String timezone) { - this.chainingStrategy = ChainingStrategy.ALWAYS; this.timezone = timezone; this.routingRules = routingRules; this.metadataApplier = metadataApplier; diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/BatchSchemaOperatorFactory.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/BatchSchemaOperatorFactory.java new file mode 100644 index 00000000000..b9fc87ff30c --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/BatchSchemaOperatorFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.runtime.operators.schema.regular; + +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.route.RouteRule; +import org.apache.flink.cdc.common.sink.MetadataApplier; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; + +import java.util.List; + +/** + * Factory to create {@link BatchSchemaOperator} with chaining strategy set for Flink 2.x + * compatibility. + */ +@Internal +public class BatchSchemaOperatorFactory extends SimpleOperatorFactory + implements OneInputStreamOperatorFactory { + + public BatchSchemaOperatorFactory( + List routingRules, MetadataApplier metadataApplier, String timezone) { + super(new BatchSchemaOperator(routingRules, metadataApplier, timezone)); + setChainingStrategy(ChainingStrategy.ALWAYS); + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/SchemaOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/SchemaOperator.java index 901ba168db0..276cd059ced 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/SchemaOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/SchemaOperator.java @@ -39,7 +39,6 @@ import org.apache.flink.runtime.operators.coordination.CoordinationResponse; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -109,7 +108,6 @@ public SchemaOperator( Duration rpcTimeOut, SchemaChangeBehavior schemaChangeBehavior, String timezone) { - this.chainingStrategy = ChainingStrategy.ALWAYS; this.rpcTimeout = rpcTimeOut; this.schemaChangeBehavior = schemaChangeBehavior; this.timezone = timezone; @@ -131,7 +129,7 @@ public void open() throws Exception { this.schemaOperatorMetrics = new SchemaOperatorMetrics( getRuntimeContext().getMetricGroup(), schemaChangeBehavior); - this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + this.subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); this.originalSchemaMap = new HashMap<>(); this.evolvedSchemaMap = new HashMap<>(); this.router = new TableIdRouter(routingRules); diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/SchemaOperatorFactory.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/SchemaOperatorFactory.java index 630acc90463..550d3f9fec6 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/SchemaOperatorFactory.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/schema/regular/SchemaOperatorFactory.java @@ -24,6 +24,7 @@ import org.apache.flink.cdc.common.sink.MetadataApplier; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; @@ -50,6 +51,7 @@ public SchemaOperatorFactory( SchemaChangeBehavior schemaChangeBehavior, String timezone) { super(new SchemaOperator(routingRules, rpcTimeout, schemaChangeBehavior, timezone)); + setChainingStrategy(ChainingStrategy.ALWAYS); this.metadataApplier = metadataApplier; this.routingRules = routingRules; this.schemaChangeBehavior = schemaChangeBehavior; diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkWriterOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkWriterOperator.java index 52900fef740..9deeb63b21f 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkWriterOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/BatchDataSinkWriterOperator.java @@ -30,7 +30,6 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.watermark.Watermark; @@ -41,6 +40,7 @@ import java.lang.reflect.Constructor; import java.lang.reflect.Field; +import java.lang.reflect.Method; /** * An operator that processes records to be written into a {@link Sink} in batch mode. @@ -78,7 +78,6 @@ public BatchDataSinkWriterOperator( this.sink = sink; this.processingTimeService = processingTimeService; this.mailboxExecutor = mailboxExecutor; - this.chainingStrategy = ChainingStrategy.ALWAYS; } @Override @@ -88,40 +87,43 @@ public void setup( Output>> output) { super.setup(containingTask, config, output); flinkWriterOperator = createFlinkWriterOperator(); - this.>>getFlinkWriterOperator() - .setup(containingTask, config, output); + invokeFlinkWriterOperatorMethod( + "setup", + new Class[] {StreamTask.class, StreamConfig.class, Output.class}, + containingTask, + config, + output); } @Override public void open() throws Exception { - this.>>getFlinkWriterOperator().open(); + invokeFlinkWriterOperatorMethod("open", new Class[0]); copySinkWriter = getFieldValue("sinkWriter"); } @Override public void initializeState(StateInitializationContext context) throws Exception { - this.>>getFlinkWriterOperator() - .initializeState(context); + invokeFlinkWriterOperatorMethod( + "initializeState", new Class[] {StateInitializationContext.class}, context); } @Override public void snapshotState(StateSnapshotContext context) throws Exception { - this.>>getFlinkWriterOperator() - .snapshotState(context); + invokeFlinkWriterOperatorMethod( + "snapshotState", new Class[] {StateSnapshotContext.class}, context); } @Override public void processWatermark(Watermark mark) throws Exception { super.processWatermark(mark); - this.>>getFlinkWriterOperator() - .processWatermark(mark); + invokeFlinkWriterOperatorMethod("processWatermark", new Class[] {Watermark.class}, mark); } @Override public void processWatermarkStatus(WatermarkStatus watermarkStatus) throws Exception { super.processWatermarkStatus(watermarkStatus); - this.>>getFlinkWriterOperator() - .processWatermarkStatus(watermarkStatus); + invokeFlinkWriterOperatorMethod( + "processWatermarkStatus", new Class[] {WatermarkStatus.class}, watermarkStatus); } @Override @@ -138,19 +140,18 @@ public void processElement(StreamRecord element) throws Exception { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - this.>>getFlinkWriterOperator() - .prepareSnapshotPreBarrier(checkpointId); + invokeFlinkWriterOperatorMethod( + "prepareSnapshotPreBarrier", new Class[] {long.class}, checkpointId); } @Override public void close() throws Exception { - this.>>getFlinkWriterOperator() - .close(); + invokeFlinkWriterOperatorMethod("close", new Class[0]); } @Override public void endInput() throws Exception { - this.getFlinkWriterOperator().endInput(); + invokeFlinkWriterOperatorMethod("endInput", new Class[0]); } // ----------------------------- Helper functions ------------------------------- @@ -168,11 +169,68 @@ private Object createFlinkWriterOperator() { .getUserCodeClassLoader() .loadClass( "org.apache.flink.streaming.runtime.operators.sink.SinkWriterOperator"); - Constructor constructor = - flinkWriterClass.getDeclaredConstructor( - Sink.class, ProcessingTimeService.class, MailboxExecutor.class); - constructor.setAccessible(true); - return constructor.newInstance(sink, processingTimeService, mailboxExecutor); + + // Try to find a constructor whose first three parameters are compatible with + // (Sink, ProcessingTimeService, MailboxExecutor). This makes the code resilient + // against Flink 2.x adding extra parameters to the constructor. + Constructor target = null; + for (Constructor c : flinkWriterClass.getDeclaredConstructors()) { + Class[] p = c.getParameterTypes(); + if (p.length >= 3 + && Sink.class.isAssignableFrom(p[0]) + && ProcessingTimeService.class.isAssignableFrom(p[1]) + && MailboxExecutor.class.isAssignableFrom(p[2])) { + target = c; + break; + } + } + + if (target == null) { + // Fallback: use the first declared constructor and best-effort argument mapping + // below. This covers Flink 2.2 where the constructor signature may have changed. + Constructor[] all = flinkWriterClass.getDeclaredConstructors(); + if (all.length == 0) { + throw new RuntimeException( + "No constructors found on SinkWriterOperator in Flink runtime"); + } + target = all[0]; + } + + target.setAccessible(true); + Class[] paramTypes = target.getParameterTypes(); + Object[] args = new Object[paramTypes.length]; + for (int i = 0; i < paramTypes.length; i++) { + Class t = paramTypes[i]; + if (Sink.class.isAssignableFrom(t)) { + args[i] = sink; + } else if (ProcessingTimeService.class.isAssignableFrom(t)) { + args[i] = processingTimeService; + } else if (MailboxExecutor.class.isAssignableFrom(t)) { + args[i] = mailboxExecutor; + } else if (t.isPrimitive()) { + if (t == boolean.class) { + args[i] = false; + } else if (t == char.class) { + args[i] = '\0'; + } else if (t == byte.class) { + args[i] = (byte) 0; + } else if (t == short.class) { + args[i] = (short) 0; + } else if (t == int.class) { + args[i] = 0; + } else if (t == long.class) { + args[i] = 0L; + } else if (t == float.class) { + args[i] = 0.0f; + } else if (t == double.class) { + args[i] = 0.0d; + } + } else { + // Best effort: pass null for any extra, unknown parameters. + args[i] = null; + } + } + return target.newInstance(args); } catch (Exception e) { throw new RuntimeException("Failed to create SinkWriterOperator in Flink", e); } @@ -204,4 +262,24 @@ private T getFieldValue(String fieldName) throws IllegalAccessException { private T getFlinkWriterOperator() { return (T) flinkWriterOperator; } + + private void invokeFlinkWriterOperatorMethod( + String methodName, Class[] parameterTypes, Object... args) { + try { + Method m = flinkWriterOperator.getClass().getDeclaredMethod(methodName, parameterTypes); + m.setAccessible(true); + m.invoke(flinkWriterOperator, args); + } catch (NoSuchMethodException e) { + // Method does not exist in this Flink version (for example, open() signature changed); + // ignore for compatibility. + return; + } catch (Exception e) { + throw new RuntimeException( + "Failed to invoke method " + + methodName + + " on wrapped flink writer operator " + + flinkWriterOperator.getClass().getName(), + e); + } + } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperator.java index 0432d1f54cf..e24b077ca03 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperator.java @@ -36,7 +36,6 @@ import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.watermark.Watermark; @@ -47,6 +46,7 @@ import java.lang.reflect.Constructor; import java.lang.reflect.Field; +import java.lang.reflect.Method; import java.util.HashSet; import java.util.Optional; import java.util.Set; @@ -97,7 +97,6 @@ public DataSinkWriterOperator( this.mailboxExecutor = mailboxExecutor; this.schemaOperatorID = schemaOperatorID; this.processedTableIds = new HashSet<>(); - this.chainingStrategy = ChainingStrategy.ALWAYS; } @Override @@ -107,8 +106,14 @@ public void setup( Output>> output) { super.setup(containingTask, config, output); flinkWriterOperator = createFlinkWriterOperator(); - this.>>getFlinkWriterOperator() - .setup(containingTask, config, output); + // Call protected AbstractStreamOperator.setup(...) on the wrapped operator via reflection + // to avoid JVM verifier issues with protected access across packages. + invokeFlinkWriterOperatorMethod( + "setup", + new Class[] {StreamTask.class, StreamConfig.class, Output.class}, + containingTask, + config, + output); schemaEvolutionClient = new SchemaEvolutionClient( containingTask.getEnvironment().getOperatorCoordinatorEventGateway(), @@ -117,35 +122,82 @@ public void setup( @Override public void open() throws Exception { - this.>>getFlinkWriterOperator().open(); + invokeFlinkWriterOperatorMethod("open", new Class[0]); copySinkWriter = getFieldValue("sinkWriter"); + ensureEmitDownstream(); + } + + /** + * Flink 2.2 {@code SinkWriterOperator} sets {@code emitDownstream = sink instanceof + * SupportsCommitter}. Sinks like Paimon implement the older {@code TwoPhaseCommittingSink} + * rather than Flink 2.x {@code SupportsCommitter}, so committables are silently discarded. This + * method forces the flag to {@code true} and fills in the committable serializer so that the + * wrapped operator properly emits committables to the downstream committer. + */ + private void ensureEmitDownstream() { + try { + Field emitField = findField(flinkWriterOperator.getClass(), "emitDownstream"); + if (emitField == null) { + return; + } + emitField.setAccessible(true); + if (!emitField.getBoolean(flinkWriterOperator)) { + emitField.setBoolean(flinkWriterOperator, true); + + Field serField = findField(flinkWriterOperator.getClass(), "committableSerializer"); + if (serField != null) { + serField.setAccessible(true); + if (serField.get(flinkWriterOperator) == null) { + try { + Method getSerializer = + sink.getClass().getMethod("getCommittableSerializer"); + getSerializer.setAccessible(true); + serField.set(flinkWriterOperator, getSerializer.invoke(sink)); + } catch (NoSuchMethodException ignored) { + } + } + } + } + } catch (Exception e) { + LOG.warn("Could not force emitDownstream on wrapped SinkWriterOperator", e); + } + } + + private static Field findField(Class clazz, String name) { + while (clazz != null) { + try { + return clazz.getDeclaredField(name); + } catch (NoSuchFieldException e) { + clazz = clazz.getSuperclass(); + } + } + return null; } @Override public void initializeState(StateInitializationContext context) throws Exception { - schemaEvolutionClient.registerSubtask(getRuntimeContext().getIndexOfThisSubtask()); - this.>>getFlinkWriterOperator() - .initializeState(context); + schemaEvolutionClient.registerSubtask(getSubtaskIndexCompat()); + invokeFlinkWriterOperatorMethod( + "initializeState", new Class[] {StateInitializationContext.class}, context); } @Override public void snapshotState(StateSnapshotContext context) throws Exception { - this.>>getFlinkWriterOperator() - .snapshotState(context); + invokeFlinkWriterOperatorMethod( + "snapshotState", new Class[] {StateSnapshotContext.class}, context); } @Override public void processWatermark(Watermark mark) throws Exception { super.processWatermark(mark); - this.>>getFlinkWriterOperator() - .processWatermark(mark); + invokeFlinkWriterOperatorMethod("processWatermark", new Class[] {Watermark.class}, mark); } @Override public void processWatermarkStatus(WatermarkStatus watermarkStatus) throws Exception { super.processWatermarkStatus(watermarkStatus); - this.>>getFlinkWriterOperator() - .processWatermarkStatus(watermarkStatus); + invokeFlinkWriterOperatorMethod( + "processWatermarkStatus", new Class[] {WatermarkStatus.class}, watermarkStatus); } @Override @@ -188,19 +240,18 @@ public void processElement(StreamRecord element) throws Exception { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - this.>>getFlinkWriterOperator() - .prepareSnapshotPreBarrier(checkpointId); + invokeFlinkWriterOperatorMethod( + "prepareSnapshotPreBarrier", new Class[] {long.class}, checkpointId); } @Override public void close() throws Exception { - this.>>getFlinkWriterOperator() - .close(); + invokeFlinkWriterOperatorMethod("close", new Class[0]); } @Override public void endInput() throws Exception { - this.getFlinkWriterOperator().endInput(); + invokeFlinkWriterOperatorMethod("endInput", new Class[0]); } // ----------------------------- Helper functions ------------------------------- @@ -223,7 +274,7 @@ private void handleFlushEvent(FlushEvent event) throws Exception { }); } schemaEvolutionClient.notifyFlushSuccess( - getRuntimeContext().getIndexOfThisSubtask(), event.getSourceSubTaskId()); + getSubtaskIndexCompat(), event.getSourceSubTaskId()); } private void emitLatestSchema(TableId tableId) throws Exception { @@ -250,16 +301,106 @@ private Object createFlinkWriterOperator() { .getUserCodeClassLoader() .loadClass( "org.apache.flink.streaming.runtime.operators.sink.SinkWriterOperator"); - Constructor constructor = - flinkWriterClass.getDeclaredConstructor( - Sink.class, ProcessingTimeService.class, MailboxExecutor.class); - constructor.setAccessible(true); - return constructor.newInstance(sink, processingTimeService, mailboxExecutor); + + // Try to find a constructor whose first three parameters are compatible with + // (Sink, ProcessingTimeService, MailboxExecutor). This makes the code resilient + // against Flink 2.x adding extra parameters to the constructor. + Constructor target = null; + for (Constructor c : flinkWriterClass.getDeclaredConstructors()) { + Class[] p = c.getParameterTypes(); + if (p.length >= 3 + && Sink.class.isAssignableFrom(p[0]) + && ProcessingTimeService.class.isAssignableFrom(p[1]) + && MailboxExecutor.class.isAssignableFrom(p[2])) { + target = c; + break; + } + } + + if (target == null) { + // Fallback: use the first declared constructor and best-effort argument mapping + // below. This covers Flink 2.2 where the constructor signature may have changed. + Constructor[] all = flinkWriterClass.getDeclaredConstructors(); + if (all.length == 0) { + throw new RuntimeException( + "No constructors found on SinkWriterOperator in Flink runtime"); + } + target = all[0]; + } + + target.setAccessible(true); + Class[] paramTypes = target.getParameterTypes(); + Object[] args = new Object[paramTypes.length]; + for (int i = 0; i < paramTypes.length; i++) { + Class t = paramTypes[i]; + if (Sink.class.isAssignableFrom(t)) { + args[i] = sink; + } else if (ProcessingTimeService.class.isAssignableFrom(t)) { + args[i] = processingTimeService; + } else if (MailboxExecutor.class.isAssignableFrom(t)) { + args[i] = mailboxExecutor; + } else if (t.isPrimitive()) { + if (t == boolean.class) { + args[i] = false; + } else if (t == char.class) { + args[i] = '\0'; + } else if (t == byte.class) { + args[i] = (byte) 0; + } else if (t == short.class) { + args[i] = (short) 0; + } else if (t == int.class) { + args[i] = 0; + } else if (t == long.class) { + args[i] = 0L; + } else if (t == float.class) { + args[i] = 0.0f; + } else if (t == double.class) { + args[i] = 0.0d; + } + } else { + // Best effort: pass null for any extra, unknown parameters. + args[i] = null; + } + } + return target.newInstance(args); } catch (Exception e) { throw new RuntimeException("Failed to create SinkWriterOperator in Flink", e); } } + /** + * Obtains the subtask index in a way that is compatible with both Flink 1.x and 2.x. + * + *

Flink 2.x removed {@code StreamingRuntimeContext.getIndexOfThisSubtask()}. The correct + * replacement is {@code getRuntimeContext().getTaskInfo().getIndexOfThisSubtask()}. + * + *

IMPORTANT: Returning a wrong (fixed) index here causes ALL sink subtasks to report + * the same ID to SchemaCoordinator. The coordinator then waits forever for the missing subtask + * IDs, causing a 3-minute timeout in SchemaOperator. + */ + private int getSubtaskIndexCompat() { + try { + Object ctx = getRuntimeContext(); + // Flink 2.x: getTaskInfo().getIndexOfThisSubtask() + try { + Method getTaskInfo = ctx.getClass().getMethod("getTaskInfo"); + getTaskInfo.setAccessible(true); + Object taskInfo = getTaskInfo.invoke(ctx); + Method getIndex = taskInfo.getClass().getMethod("getIndexOfThisSubtask"); + getIndex.setAccessible(true); + return (Integer) getIndex.invoke(taskInfo); + } catch (NoSuchMethodException ignored) { + // fall through to Flink 1.x path + } + // Flink 1.x: direct getIndexOfThisSubtask() + Method m = ctx.getClass().getMethod("getIndexOfThisSubtask"); + m.setAccessible(true); + return (Integer) m.invoke(ctx); + } catch (Exception e) { + throw new RuntimeException("Failed to obtain subtask index from RuntimeContext", e); + } + } + /** * Finds a field by name from its declaring class. This also searches for the field in super * classes. @@ -286,4 +427,24 @@ private T getFieldValue(String fieldName) throws IllegalAccessException { private T getFlinkWriterOperator() { return (T) flinkWriterOperator; } + + private void invokeFlinkWriterOperatorMethod( + String methodName, Class[] parameterTypes, Object... args) { + try { + Method m = flinkWriterOperator.getClass().getDeclaredMethod(methodName, parameterTypes); + m.setAccessible(true); + m.invoke(flinkWriterOperator, args); + } catch (NoSuchMethodException e) { + // Method does not exist in this Flink version (for example, open() signature changed); + // ignore for compatibility. + return; + } catch (Exception e) { + throw new RuntimeException( + "Failed to invoke method " + + methodName + + " on wrapped flink writer operator " + + flinkWriterOperator.getClass().getName(), + e); + } + } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperatorFactory.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperatorFactory.java index 278518f976e..a286ac2e70c 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperatorFactory.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/DataSinkWriterOperatorFactory.java @@ -23,17 +23,16 @@ import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorParameters; -import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; /** Operator factory for {@link DataSinkWriterOperator}. */ @Internal public class DataSinkWriterOperatorFactory extends AbstractStreamOperatorFactory> - implements OneInputStreamOperatorFactory>, - YieldingOperatorFactory> { + implements OneInputStreamOperatorFactory> { private final Sink sink; private final boolean isBounded; @@ -41,6 +40,7 @@ public class DataSinkWriterOperatorFactory public DataSinkWriterOperatorFactory( Sink sink, boolean isBounded, OperatorID schemaOperatorID) { + setChainingStrategy(ChainingStrategy.ALWAYS); this.sink = sink; this.isBounded = isBounded; this.schemaOperatorID = schemaOperatorID; @@ -50,11 +50,10 @@ public DataSinkWriterOperatorFactory( @Override public >> T createStreamOperator( StreamOperatorParameters> parameters) { - if (isBounded) { BatchDataSinkWriterOperator writerOperator = new BatchDataSinkWriterOperator<>( - sink, processingTimeService, getMailboxExecutor()); + sink, processingTimeService, parameters.getMailboxExecutor()); writerOperator.setup( parameters.getContainingTask(), parameters.getStreamConfig(), @@ -63,7 +62,10 @@ public >> T createStreamOpera } DataSinkWriterOperator writerOperator = new DataSinkWriterOperator<>( - sink, processingTimeService, getMailboxExecutor(), schemaOperatorID); + sink, + processingTimeService, + parameters.getMailboxExecutor(), + schemaOperatorID); writerOperator.setup( parameters.getContainingTask(), parameters.getStreamConfig(), diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/exception/SinkWrapperException.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/exception/SinkWrapperException.java index 18dab43dd8b..273a6fb8cce 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/exception/SinkWrapperException.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/sink/exception/SinkWrapperException.java @@ -20,9 +20,9 @@ import org.apache.flink.cdc.common.event.Event; /** - * A generic {@link RuntimeException} thrown when an error occurs in {@link - * org.apache.flink.cdc.runtime.operators.sink.DataSinkWriterOperator} and {@link - * org.apache.flink.cdc.runtime.operators.sink.DataSinkFunctionOperator}. + * A generic {@link RuntimeException} thrown when an error occurs in sink operators (e.g. {@link + * org.apache.flink.cdc.runtime.operators.sink.DataSinkWriterOperator} or the SinkFunction-based + * operator in the Flink 1.x compat module). */ public class SinkWrapperException extends RuntimeException { public SinkWrapperException(Event event, Throwable cause) { diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java index ff6cc512f44..235c5fc64bc 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperator.java @@ -39,7 +39,6 @@ import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -85,7 +84,6 @@ public static PreTransformOperatorBuilder newBuilder() { this.preTransformChangeInfoMap = new HashMap<>(); this.preTransformProcessorMap = new HashMap<>(); this.schemaMetadataTransformers = new ArrayList<>(); - this.chainingStrategy = ChainingStrategy.ALWAYS; this.transformRules = transformRules; this.udfFunctions = udfFunctions; diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperatorBuilder.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperatorBuilder.java index bb4df65a5f6..32b12617183 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperatorBuilder.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperatorBuilder.java @@ -79,4 +79,10 @@ public PreTransformOperatorBuilder addUdfFunctions( public PreTransformOperator build() { return new PreTransformOperator(transformRules, udfFunctions); } + + /** Builds a factory with chaining strategy set (for Flink 2.x compatibility). */ + public PreTransformOperatorFactory buildFactory() { + return new PreTransformOperatorFactory( + new PreTransformOperator(transformRules, udfFunctions)); + } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperatorFactory.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperatorFactory.java new file mode 100644 index 00000000000..fea98e2d5f2 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/operators/transform/PreTransformOperatorFactory.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.runtime.operators.transform; + +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; + +/** + * Factory for {@link PreTransformOperator} with chaining strategy set for Flink 2.x compatibility. + */ +@Internal +public class PreTransformOperatorFactory extends SimpleOperatorFactory + implements OneInputStreamOperatorFactory { + + public PreTransformOperatorFactory(PreTransformOperator operator) { + super(operator); + setChainingStrategy(ChainingStrategy.ALWAYS); + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/BatchRegularPrePartitionOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/BatchRegularPrePartitionOperator.java index d6142868f17..90901a7b1e0 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/BatchRegularPrePartitionOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/BatchRegularPrePartitionOperator.java @@ -29,7 +29,6 @@ import org.apache.flink.cdc.runtime.serializer.event.EventSerializer; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -58,7 +57,6 @@ public class BatchRegularPrePartitionOperator extends AbstractStreamOperator hashFunctionProvider) { - this.chainingStrategy = ChainingStrategy.ALWAYS; this.downstreamParallelism = downstreamParallelism; this.hashFunctionProvider = hashFunctionProvider; } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/BatchRegularPrePartitionOperatorFactory.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/BatchRegularPrePartitionOperatorFactory.java new file mode 100644 index 00000000000..4f9ad2288bc --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/BatchRegularPrePartitionOperatorFactory.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.runtime.partitioning; + +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.function.HashFunctionProvider; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; + +/** + * Factory for {@link BatchRegularPrePartitionOperator} with chaining strategy set for Flink 2.x + * compatibility. + */ +@Internal +public class BatchRegularPrePartitionOperatorFactory + extends SimpleOperatorFactory + implements OneInputStreamOperatorFactory { + + public BatchRegularPrePartitionOperatorFactory( + int downstreamParallelism, HashFunctionProvider hashFunctionProvider) { + super(new BatchRegularPrePartitionOperator(downstreamParallelism, hashFunctionProvider)); + setChainingStrategy(ChainingStrategy.ALWAYS); + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/DistributedPrePartitionOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/DistributedPrePartitionOperator.java index bedbe8e8e09..02a9469bdb8 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/DistributedPrePartitionOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/DistributedPrePartitionOperator.java @@ -30,7 +30,6 @@ import org.apache.flink.cdc.runtime.serializer.event.EventSerializer; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -55,7 +54,6 @@ public class DistributedPrePartitionOperator extends AbstractStreamOperator hashFunctionProvider) { - this.chainingStrategy = ChainingStrategy.ALWAYS; this.downstreamParallelism = downstreamParallelism; this.hashFunctionProvider = hashFunctionProvider; } @@ -63,7 +61,7 @@ public DistributedPrePartitionOperator( @Override public void open() throws Exception { super.open(); - subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); schemaMap = new HashMap<>(); hashFunctionMap = new HashMap<>(); } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/DistributedPrePartitionOperatorFactory.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/DistributedPrePartitionOperatorFactory.java new file mode 100644 index 00000000000..3a90065e51e --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/DistributedPrePartitionOperatorFactory.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.runtime.partitioning; + +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.function.HashFunctionProvider; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; + +/** + * Factory for {@link DistributedPrePartitionOperator} with chaining strategy set for Flink 2.x + * compatibility. + */ +@Internal +public class DistributedPrePartitionOperatorFactory extends SimpleOperatorFactory + implements OneInputStreamOperatorFactory { + + public DistributedPrePartitionOperatorFactory( + int downstreamParallelism, HashFunctionProvider hashFunctionProvider) { + super(new DistributedPrePartitionOperator(downstreamParallelism, hashFunctionProvider)); + setChainingStrategy(ChainingStrategy.ALWAYS); + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/RegularPrePartitionOperator.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/RegularPrePartitionOperator.java index 38f177472fb..106d189a9c9 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/RegularPrePartitionOperator.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/RegularPrePartitionOperator.java @@ -33,7 +33,6 @@ import org.apache.flink.runtime.jobgraph.tasks.TaskOperatorEventGateway; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -67,7 +66,6 @@ public RegularPrePartitionOperator( OperatorID schemaOperatorId, int downstreamParallelism, HashFunctionProvider hashFunctionProvider) { - this.chainingStrategy = ChainingStrategy.ALWAYS; this.schemaOperatorId = schemaOperatorId; this.downstreamParallelism = downstreamParallelism; this.hashFunctionProvider = hashFunctionProvider; diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/RegularPrePartitionOperatorFactory.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/RegularPrePartitionOperatorFactory.java new file mode 100644 index 00000000000..eef12bdf116 --- /dev/null +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/partitioning/RegularPrePartitionOperatorFactory.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.cdc.runtime.partitioning; + +import org.apache.flink.cdc.common.annotation.Internal; +import org.apache.flink.cdc.common.event.DataChangeEvent; +import org.apache.flink.cdc.common.event.Event; +import org.apache.flink.cdc.common.function.HashFunctionProvider; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; + +/** + * Factory for {@link RegularPrePartitionOperator} with chaining strategy set for Flink 2.x + * compatibility. + */ +@Internal +public class RegularPrePartitionOperatorFactory extends SimpleOperatorFactory + implements OneInputStreamOperatorFactory { + + public RegularPrePartitionOperatorFactory( + OperatorID schemaOperatorId, + int downstreamParallelism, + HashFunctionProvider hashFunctionProvider) { + super( + new RegularPrePartitionOperator( + schemaOperatorId, downstreamParallelism, hashFunctionProvider)); + setChainingStrategy(ChainingStrategy.ALWAYS); + } +} diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/EnumSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/EnumSerializer.java index 5b93c320d4f..c7828108960 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/EnumSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/EnumSerializer.java @@ -238,7 +238,8 @@ public TypeSerializer restoreSerializer() { return new EnumSerializer<>(enumClass, previousEnums); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializer newSerializer) { if (!(newSerializer instanceof EnumSerializer)) { @@ -269,6 +270,32 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.compatibleWithReconfiguredSerializer( reconfiguredSerializer); } + + // Flink 2.x new abstract method - no @Override so this also compiles against Flink 1.x + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof EnumSerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + @SuppressWarnings("unchecked") + EnumSerializerSnapshot oldSnapshot = + (EnumSerializerSnapshot) oldSerializerSnapshot; + if (!enumClass.equals(oldSnapshot.enumClass)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + if (Arrays.equals(oldSnapshot.previousEnums, previousEnums)) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + Set reconfiguredEnumSet = + new LinkedHashSet<>(Arrays.asList(oldSnapshot.previousEnums)); + reconfiguredEnumSet.addAll(Arrays.asList(previousEnums)); + @SuppressWarnings("unchecked") + T[] reconfiguredEnums = + reconfiguredEnumSet.toArray( + (T[]) Array.newInstance(enumClass, reconfiguredEnumSet.size())); + return TypeSerializerSchemaCompatibility.compatibleWithReconfiguredSerializer( + new EnumSerializer<>(enumClass, reconfiguredEnums)); + } } // -------------------------------------------------------------------------------------------- diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NestedSerializersSnapshotDelegate.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NestedSerializersSnapshotDelegate.java index 66c62630dac..8eedaeef14e 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NestedSerializersSnapshotDelegate.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NestedSerializersSnapshotDelegate.java @@ -184,7 +184,14 @@ public static NestedSerializersSnapshotDelegate readNestedSerializerSnapshots( // Utilities // ------------------------------------------------------------------------ - /** Utility method to conjure up a new scope for the generic parameters. */ + /** + * Utility method to conjure up a new scope for the generic parameters. + * + *

In Flink 1.x, TypeSerializerSnapshot.resolveSchemaCompatibility(TypeSerializer) is the + * abstract method. In Flink 2.x, it was replaced by + * resolveSchemaCompatibility(TypeSerializerSnapshot). This method uses reflection to call the + * right variant at runtime. + */ @SuppressWarnings("unchecked") private static TypeSerializerSchemaCompatibility resolveCompatibility( TypeSerializer serializer, TypeSerializerSnapshot snapshot) { @@ -192,7 +199,32 @@ private static TypeSerializerSchemaCompatibility resolveCompatibility( TypeSerializer typedSerializer = (TypeSerializer) serializer; TypeSerializerSnapshot typedSnapshot = (TypeSerializerSnapshot) snapshot; - return typedSnapshot.resolveSchemaCompatibility(typedSerializer); + // Try Flink 1.x approach: oldSnapshot.resolveSchemaCompatibility(newSerializer) + try { + java.lang.reflect.Method m = + typedSnapshot + .getClass() + .getMethod("resolveSchemaCompatibility", TypeSerializer.class); + return (TypeSerializerSchemaCompatibility) m.invoke(typedSnapshot, typedSerializer); + } catch (NoSuchMethodException ignored) { + // fall through to Flink 2.x approach + } catch (Exception e) { + throw new RuntimeException( + "Failed to resolve schema compatibility (Flink 1.x path)", e); + } + + // Flink 2.x approach: newSnapshot.resolveSchemaCompatibility(oldSnapshot) + try { + TypeSerializerSnapshot newSnapshot = typedSerializer.snapshotConfiguration(); + java.lang.reflect.Method m = + newSnapshot + .getClass() + .getMethod("resolveSchemaCompatibility", TypeSerializerSnapshot.class); + return (TypeSerializerSchemaCompatibility) m.invoke(newSnapshot, typedSnapshot); + } catch (Exception e) { + throw new RuntimeException( + "Failed to resolve schema compatibility (Flink 2.x path)", e); + } } private static TypeSerializer[] snapshotsToRestoreSerializers( diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NullableSerializerWrapper.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NullableSerializerWrapper.java index c230c8c9a8c..33eb92742fd 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NullableSerializerWrapper.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/NullableSerializerWrapper.java @@ -172,7 +172,8 @@ public TypeSerializer restoreSerializer() { return new NullableSerializerWrapper<>(innerSerializer); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializer newSerializer) { if (!(newSerializer instanceof NullableSerializerWrapper)) { @@ -187,5 +188,20 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.compatibleAsIs(); } } + + // Flink 2.x new abstract method - no @Override so this also compiles against Flink 1.x + @SuppressWarnings("unchecked") + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof NullableSerializerWrapperSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + NullableSerializerWrapperSnapshot snapshot = + (NullableSerializerWrapperSnapshot) oldSerializerSnapshot; + if (!innerSerializer.equals(snapshot.innerSerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ArrayDataSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ArrayDataSerializer.java index 39939375d50..ecbc03d7e9d 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ArrayDataSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ArrayDataSerializer.java @@ -265,7 +265,8 @@ public TypeSerializer restoreSerializer() { return new ArrayDataSerializer(previousType, previousEleSer); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializer newSerializer) { if (!(newSerializer instanceof ArrayDataSerializer)) { @@ -280,5 +281,20 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.compatibleAsIs(); } } + + // Flink 2.x new abstract method - no @Override so this also compiles against Flink 1.x + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof ArrayDataSerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + ArrayDataSerializerSnapshot snapshot = + (ArrayDataSerializerSnapshot) oldSerializerSnapshot; + if (!previousType.equals(snapshot.previousType) + || !previousEleSer.equals(snapshot.previousEleSer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/DecimalDataSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/DecimalDataSerializer.java index 4872330f37e..97d2e356147 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/DecimalDataSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/DecimalDataSerializer.java @@ -178,7 +178,8 @@ public TypeSerializer restoreSerializer() { return new DecimalDataSerializer(previousPrecision, previousScale); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializer newSerializer) { if (!(newSerializer instanceof DecimalDataSerializer)) { @@ -193,5 +194,21 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility return TypeSerializerSchemaCompatibility.compatibleAsIs(); } } + + // Flink 2.x changed the abstract method to + // resolveSchemaCompatibility(TypeSerializerSnapshot). + // No @Override so this also compiles against Flink 1.x where the method does not exist. + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof DecimalSerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + DecimalSerializerSnapshot snapshot = (DecimalSerializerSnapshot) oldSerializerSnapshot; + if (previousPrecision != snapshot.previousPrecision + || previousScale != snapshot.previousScale) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/LocalZonedTimestampDataSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/LocalZonedTimestampDataSerializer.java index c3622db2340..987f2082353 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/LocalZonedTimestampDataSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/LocalZonedTimestampDataSerializer.java @@ -177,7 +177,8 @@ public TypeSerializer restoreSerializer() { return new LocalZonedTimestampDataSerializer(previousPrecision); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility(TypeSerializer newSerializer) { if (!(newSerializer instanceof LocalZonedTimestampDataSerializer)) { @@ -192,5 +193,22 @@ public TypeSerializer restoreSerializer() { return TypeSerializerSchemaCompatibility.compatibleAsIs(); } } + + // Flink 2.x changed the abstract method to + // resolveSchemaCompatibility(TypeSerializerSnapshot). + // No @Override so this also compiles against Flink 1.x where the method does not exist. + public TypeSerializerSchemaCompatibility + resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof LocalZonedTimestampDataSerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + LocalZonedTimestampDataSerializerSnapshot snapshot = + (LocalZonedTimestampDataSerializerSnapshot) oldSerializerSnapshot; + if (previousPrecision != snapshot.previousPrecision) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/MapDataSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/MapDataSerializer.java index d8f92ee29f0..8b2e19cdef7 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/MapDataSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/MapDataSerializer.java @@ -306,7 +306,8 @@ public TypeSerializer restoreSerializer() { return new MapDataSerializer(keyType, valueType, keySerializer, valueSerializer); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializer newSerializer) { if (!(newSerializer instanceof MapDataSerializer)) { @@ -323,5 +324,21 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.compatibleAsIs(); } } + + // Flink 2.x new abstract method - no @Override so this also compiles against Flink 1.x + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof MapDataSerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + MapDataSerializerSnapshot snapshot = (MapDataSerializerSnapshot) oldSerializerSnapshot; + if (!keyType.equals(snapshot.keyType) + || !valueType.equals(snapshot.valueType) + || !keySerializer.equals(snapshot.keySerializer) + || !valueSerializer.equals(snapshot.valueSerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/TimestampDataSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/TimestampDataSerializer.java index babc6345bc5..10dfc060631 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/TimestampDataSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/TimestampDataSerializer.java @@ -171,7 +171,8 @@ public TypeSerializer restoreSerializer() { return new TimestampDataSerializer(previousPrecision); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializer newSerializer) { if (!(newSerializer instanceof TimestampDataSerializer)) { @@ -186,5 +187,21 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibili return TypeSerializerSchemaCompatibility.compatibleAsIs(); } } + + // Flink 2.x changed the abstract method to + // resolveSchemaCompatibility(TypeSerializerSnapshot). + // No @Override so this also compiles against Flink 1.x where the method does not exist. + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof TimestampDataSerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + TimestampDataSerializerSnapshot snapshot = + (TimestampDataSerializerSnapshot) oldSerializerSnapshot; + if (previousPrecision != snapshot.previousPrecision) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ZonedTimestampDataSerializer.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ZonedTimestampDataSerializer.java index ca9861761cf..8676d903f75 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ZonedTimestampDataSerializer.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/serializer/data/ZonedTimestampDataSerializer.java @@ -181,7 +181,8 @@ public TypeSerializer restoreSerializer() { return new ZonedTimestampDataSerializer(previousPrecision); } - @Override + // Flink 1.x abstract method - removed in Flink 2.x, @Override omitted for cross-version + // compatibility public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializer newSerializer) { if (!(newSerializer instanceof ZonedTimestampDataSerializer)) { @@ -196,5 +197,19 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompat return TypeSerializerSchemaCompatibility.compatibleAsIs(); } } + + // Flink 2.x new abstract method - no @Override so this also compiles against Flink 1.x + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof ZonedTimestampDataSerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + ZonedTimestampDataSerializerSnapshot snapshot = + (ZonedTimestampDataSerializerSnapshot) oldSerializerSnapshot; + if (previousPrecision != snapshot.previousPrecision) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } } } diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/EventTypeInfo.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/EventTypeInfo.java index 8087746cdec..c126abca628 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/EventTypeInfo.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/EventTypeInfo.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.runtime.typeutils; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.cdc.common.event.Event; @@ -58,11 +59,18 @@ public boolean isKeyType() { return false; } - @Override + // Flink 1.x abstract method - not abstract in Flink 2.x so @Override omitted for cross-version + // compatibility public TypeSerializer createSerializer(ExecutionConfig config) { return EventSerializer.INSTANCE; } + /** For Flink 2.2+ (TypeInformation.createSerializer now takes SerializerConfig). */ + @Override + public TypeSerializer createSerializer(SerializerConfig config) { + return EventSerializer.INSTANCE; + } + @Override public String toString() { return "Event"; diff --git a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/PartitioningEventTypeInfo.java b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/PartitioningEventTypeInfo.java index 4a3018fb98a..1de112319e4 100644 --- a/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/PartitioningEventTypeInfo.java +++ b/flink-cdc-runtime/src/main/java/org/apache/flink/cdc/runtime/typeutils/PartitioningEventTypeInfo.java @@ -18,6 +18,7 @@ package org.apache.flink.cdc.runtime.typeutils; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.cdc.common.annotation.Internal; @@ -58,11 +59,18 @@ public boolean isKeyType() { return false; } - @Override + // Flink 1.x abstract method - not abstract in Flink 2.x so @Override omitted for cross-version + // compatibility public TypeSerializer createSerializer(ExecutionConfig config) { return PartitioningEventSerializer.INSTANCE; } + /** For Flink 2.2+ (TypeInformation.createSerializer now takes SerializerConfig). */ + @Override + public TypeSerializer createSerializer(SerializerConfig config) { + return PartitioningEventSerializer.INSTANCE; + } + @Override public String toString() { return "PartitioningEvent"; diff --git a/pom.xml b/pom.xml index 3d87b69054f..8f021aaffa5 100644 --- a/pom.xml +++ b/pom.xml @@ -39,6 +39,7 @@ limitations under the License. flink-cdc-dist flink-cdc-connect flink-cdc-runtime + flink-cdc-flink-compat flink-cdc-e2e-tests flink-cdc-pipeline-udf-examples flink-cdc-pipeline-model @@ -106,6 +107,8 @@ limitations under the License. and thus is changing back to java 1.6 on each maven re-import --> ${source.java.version} ${target.java.version} + + flink-cdc-flink-compat-flink1 @@ -732,6 +735,18 @@ limitations under the License. + + + flink-2.2 + + 2.2.0 + 2.2 + flink-cdc-flink-compat-flink2 + 17 + 17 + 17 + + fast