Skip to content
Draft
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
2 changes: 1 addition & 1 deletion .github/workflows/flink_cdc_base.yml
Original file line number Diff line number Diff line change
Expand Up @@ -108,7 +108,7 @@ jobs:
build_maven_parameter="${build_maven_parameter:+$build_maven_parameter }${{ inputs.custom-maven-parameter }}"
mvn --no-snapshot-updates -B -DskipTests -pl $compile_modules -am install && mvn --no-snapshot-updates -B $build_maven_parameter -pl $modules -DspecifiedParallelism=${{ inputs.parallelism }} -Duser.timezone=$jvm_timezone verify
mvn --no-snapshot-updates -B -DskipTests ${{ inputs.custom-maven-parameter }} -pl $compile_modules -am install && mvn --no-snapshot-updates -B $build_maven_parameter -pl $modules -DspecifiedParallelism=${{ inputs.parallelism }} -Duser.timezone=$jvm_timezone verify
- name: Print JVM thread dumps when cancelled
if: ${{ failure() }}
Expand Down
22 changes: 22 additions & 0 deletions .github/workflows/flink_cdc_ci.yml
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,14 @@ jobs:
with:
java-versions: "[11]"
modules: "['core']"
common_2_x:
name: Common Unit Tests 2.x
uses: ./.github/workflows/flink_cdc_base.yml
with:
java-versions: "[11]"
flink-versions: "['2.2.0']"
custom-maven-parameter: "-Pflink2"
modules: "['core_2.x']"
pipeline-ut:
name: Pipeline Unit Tests
uses: ./.github/workflows/flink_cdc_base.yml
Expand All @@ -88,6 +96,20 @@ jobs:
flink-versions: "['1.19.3', '1.20.3']"
modules: "['pipeline_e2e']"
parallelism: ${{ matrix.parallelism }}
pipeline_e2e_2_x:
strategy:
max-parallel: 2
fail-fast: false
matrix:
parallelism: [ 1, 4 ]
name: Pipeline E2E Tests 2.x (${{ matrix.parallelism }}-Parallelism)
uses: ./.github/workflows/flink_cdc_base.yml
with:
java-versions: "[11]"
flink-versions: "['2.2.0']"
custom-maven-parameter: "-Pflink2"
modules: "['pipeline_e2e_2.x']"
parallelism: ${{ matrix.parallelism }}
source_e2e:
name: Source E2E Tests
uses: ./.github/workflows/flink_cdc_base.yml
Expand Down
24 changes: 24 additions & 0 deletions .github/workflows/flink_cdc_ci_nightly.yml
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,15 @@ jobs:
with:
java-versions: "[17]"
modules: "['core']"
common_2_x:
if: github.repository == 'apache/flink-cdc'
name: Common Unit Tests 2.x
uses: ./.github/workflows/flink_cdc_base.yml
with:
java-versions: "[17]"
flink-versions: "['2.2.0']"
custom-maven-parameter: "-Pflink2"
modules: "['core_2.x']"
pipeline-ut:
if: github.repository == 'apache/flink-cdc'
name: Pipeline Unit Tests
Expand Down Expand Up @@ -82,6 +91,21 @@ jobs:
flink-versions: "['1.19.3', '1.20.3']"
modules: "['pipeline_e2e']"
parallelism: ${{ matrix.parallelism }}
pipeline_e2e_2_x:
if: github.repository == 'apache/flink-cdc'
strategy:
max-parallel: 2
fail-fast: false
matrix:
parallelism: [ 1, 4 ]
name: Pipeline E2E Tests 2.x (${{ matrix.parallelism }}-Parallelism)
uses: ./.github/workflows/flink_cdc_base.yml
with:
java-versions: "[17]"
custom-maven-parameter: "-Pflink2"
flink-versions: "['2.2.0']"
modules: "['pipeline_e2e_2.x']"
parallelism: ${{ matrix.parallelism }}
source_e2e:
if: github.repository == 'apache/flink-cdc'
name: Source E2E Tests
Expand Down
23 changes: 21 additions & 2 deletions .github/workflows/modules.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,15 @@
"flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values"
]

MODULES_CORE_2_X = [
"flink-cdc-cli",
"flink-cdc-common",
"flink-cdc-composer",
"flink-cdc-runtime",
"flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values",
"flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-values-2.x"
]

MODULES_PIPELINE_CONNECTORS = [
"flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-doris",
"flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-elasticsearch",
Expand Down Expand Up @@ -136,12 +145,17 @@
"flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests"
]

MODULES_PIPELINE_E2E_2_X = [
"flink-cdc-e2e-tests/flink-cdc-pipeline-e2e-tests-2.x"
]

MODULES_SOURCE_E2E = [
"flink-cdc-e2e-tests/flink-cdc-source-e2e-tests"
]

ALL_MODULES = set(
MODULES_CORE +
MODULES_CORE_2_X +
MODULES_PIPELINE_CONNECTORS +
MODULES_MYSQL_SOURCE +
MODULES_MYSQL_PIPELINE +
Expand Down Expand Up @@ -169,13 +183,18 @@
MODULES_SOURCE_E2E
)

# Modules that require the flink2 Maven profile to be activated
ALL_MODULES_FLINK2 = ALL_MODULES | set(MODULES_PIPELINE_E2E_2_X)

test_modules = set()
compile_modules = set()

for module in INPUT_MODULES.split(', '):
module_list = set(globals()['MODULES_' + module.upper().replace('-', '_')])
module_list = set(globals()['MODULES_' + module.upper().replace('-', '_').replace('.', '_')])
test_modules |= module_list
if module == 'source_e2e' or module == 'pipeline_e2e':
if module == 'pipeline_e2e_2.x':
compile_modules |= ALL_MODULES_FLINK2
elif module == 'source_e2e' or module == 'pipeline_e2e':
compile_modules |= ALL_MODULES
else:
compile_modules |= module_list
Expand Down
26 changes: 26 additions & 0 deletions flink-cdc-cli/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -82,4 +82,30 @@ limitations under the License.

</dependencies>

<profiles>
<profile>
<id>flink2</id>
<properties>
<flink.version>${flink.2.x.version}</flink.version>
</properties>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-dependency-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<additionalClasspathElements>
<additionalClasspathElement>${project.build.directory}/flink2-extra-libs/flink-shaded-guava-${flink.2.x.shaded.guava.version}.jar</additionalClasspathElement>
</additionalClasspathElements>
</configuration>
</plugin>
</plugins>
</build>
</profile>
</profiles>

</project>
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import org.apache.flink.cdc.composer.PipelineExecution;
import org.apache.flink.configuration.DeploymentOptions;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.jobgraph.SavepointConfigOptions;
import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;

import org.apache.flink.shaded.guava31.com.google.common.base.Joiner;
Expand Down Expand Up @@ -173,7 +172,12 @@ private static SavepointRestoreSettings createSavepointRestoreSettings(
commandLine.getOptionValue(SAVEPOINT_CLAIM_MODE),
ConfigurationUtils.getClaimModeClass());
} else {
restoreMode = SavepointConfigOptions.RESTORE_MODE.defaultValue();
try {
restoreMode =
ConfigurationUtils.getClaimModeClass().getField("DEFAULT").get(null);
} catch (NoSuchFieldException | IllegalAccessException e) {
throw new RuntimeException("Failed to get default restore mode.", e);
}
}
// allowNonRestoredState is always false because all operators are predefined.

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,11 @@ public static Class<?> getClaimModeClass() {
try {
return Class.forName("org.apache.flink.runtime.jobgraph.RestoreMode");
} catch (ClassNotFoundException e) {
throw new RuntimeException(e);
try {
return Class.forName("org.apache.flink.core.execution.RecoveryClaimMode");
} catch (ClassNotFoundException classNotFoundException) {
throw new RuntimeException(classNotFoundException);
}
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,10 @@

package org.apache.flink.cdc.cli;

import org.apache.flink.cdc.cli.utils.ConfigurationUtils;
import org.apache.flink.cdc.composer.PipelineComposer;
import org.apache.flink.cdc.composer.PipelineExecution;
import org.apache.flink.cdc.composer.definition.PipelineDef;
import org.apache.flink.core.execution.RestoreMode;
import org.apache.flink.core.fs.Path;

import org.apache.flink.shaded.guava31.com.google.common.io.Resources;
Expand Down Expand Up @@ -107,7 +107,8 @@ void testSavePointConfiguration() throws Exception {
"-n");
assertThat(executor.getFlinkConfig().get(SAVEPOINT_PATH))
.isEqualTo(flinkHome() + "/savepoints/savepoint-1");
assertThat(executor.getFlinkConfig().get(RESTORE_MODE)).isEqualTo(RestoreMode.NO_CLAIM);
assertThat(executor.getFlinkConfig().get(RESTORE_MODE))
.isEqualTo(getRestoreModeEnum("NO_CLAIM"));
assertThat(executor.getFlinkConfig().get(SAVEPOINT_IGNORE_UNCLAIMED_STATE)).isTrue();
}

Expand Down Expand Up @@ -225,6 +226,10 @@ private CliExecutor createExecutor(String... args) throws Exception {
return CliFrontend.createExecutor(parser.parse(cliOptions, args));
}

private Object getRestoreModeEnum(String name) throws Exception {
return Enum.valueOf((Class<Enum>) ConfigurationUtils.getClaimModeClass(), name);
}

private String pipelineDef() throws Exception {
URL resource = Resources.getResource("definitions/pipeline-definition-full.yaml");
return Paths.get(resource.toURI()).toString();
Expand Down
38 changes: 38 additions & 0 deletions flink-cdc-common/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -44,4 +44,42 @@ limitations under the License.
</plugins>
</build>

<profiles>
<profile>
<id>flink2</id>
<properties>
<flink.version>${flink.2.x.version}</flink.version>
</properties>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<excludes>
<!-- SinkFunction is moved to legacy package in Flink 2.x -->
<exclude>org/apache/flink/cdc/common/sink/FlinkSinkFunctionProvider.java</exclude>
<!-- SourceFunction is moved to legacy package in Flink 2.x -->
<exclude>org/apache/flink/cdc/common/source/FlinkSourceFunctionProvider.java</exclude>
</excludes>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-dependency-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<additionalClasspathElements>
<additionalClasspathElement>${project.build.directory}/flink2-extra-libs/flink-shaded-guava-${flink.2.x.shaded.guava.version}.jar</additionalClasspathElement>
</additionalClasspathElements>
</configuration>
</plugin>
</plugins>
</build>
</profile>
</profiles>

</project>
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,7 +19,6 @@

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.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@

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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.flink.cdc.common.types;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.cdc.common.annotation.PublicEvolving;

import java.util.Collections;
import java.util.List;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.flink.cdc.common.utils;

import org.apache.flink.annotation.Internal;
import org.apache.flink.cdc.common.annotation.Internal;

import java.util.LinkedHashMap;
import java.util.Map;
Expand Down
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
Loading
Loading