Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
51 changes: 51 additions & 0 deletions build-with-java17.sh
Original file line number Diff line number Diff line change
@@ -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
Original file line number Diff line number Diff line change
@@ -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);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {}
Original file line number Diff line number Diff line change
Expand Up @@ -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 {}
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down
11 changes: 11 additions & 0 deletions flink-cdc-composer/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -33,11 +33,22 @@ limitations under the License.
<artifactId>flink-cdc-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-cdc-flink-compat-api</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-cdc-runtime</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-cdc-flink-compat-flink1</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-cdc-pipeline-connector-values</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -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.
*
* <p>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<FlinkPipelineBridge> loader =
ServiceLoader.load(
FlinkPipelineBridge.class, FlinkPipelineBridge.class.getClassLoader());
Iterator<FlinkPipelineBridge> 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() {}
}
Loading