Skip to content
Merged
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
22 changes: 19 additions & 3 deletions .github/workflows/flink_cdc_ci.yml
Original file line number Diff line number Diff line change
Expand Up @@ -70,19 +70,35 @@ jobs:
java-versions: "[11]"
flink-versions: "['2.2.0']"
custom-maven-parameter: "-Pflink2"
modules: "['core_2.x']"
modules: "['core']"
pipeline-ut:
name: Pipeline Unit Tests
uses: ./.github/workflows/flink_cdc_base.yml
with:
java-versions: "[11]"
modules: "['doris, elasticsearch, fluss, hudi, iceberg, kafka, maxcompute, mysql-pipeline, oceanbase-pipeline, paimon, postgres-pipeline, starrocks, oracle-pipeline']"
pipeline-connectors-ut-2-x:
name: Pipeline Connectors 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: "['doris, elasticsearch, fluss, iceberg, kafka, maxcompute, mysql-pipeline, oceanbase-pipeline, paimon, postgres-pipeline, starrocks, oracle-pipeline']"
source-ut:
name: Source Unit Tests
uses: ./.github/workflows/flink_cdc_base.yml
with:
java-versions: "[11]"
modules: "['mysql-source', 'postgres-source, oceanbase-source, tidb, vitess', 'oracle, sqlserver', 'db2, mongodb']"
source-ut-2-x:
name: Source 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: "['mysql-source', 'postgres-source, oceanbase-source, tidb, vitess', 'oracle, sqlserver', 'db2, mongodb']"
pipeline_e2e:
strategy:
max-parallel: 2
Expand All @@ -93,7 +109,7 @@ jobs:
uses: ./.github/workflows/flink_cdc_base.yml
with:
java-versions: "[11]"
flink-versions: "['1.19.3', '1.20.3']"
flink-versions: "['1.20.3']"
modules: "['pipeline_e2e']"
parallelism: ${{ matrix.parallelism }}
pipeline_e2e_2_x:
Expand All @@ -115,5 +131,5 @@ jobs:
uses: ./.github/workflows/flink_cdc_base.yml
with:
java-versions: "[11]"
flink-versions: "['1.19.3', '1.20.3']"
flink-versions: "['1.20.3']"
modules: "['source_e2e']"
9 changes: 0 additions & 9 deletions .github/workflows/modules.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,14 +17,6 @@
"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"
]

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 @@ -150,7 +142,6 @@

ALL_MODULES = set(
MODULES_CORE +
MODULES_CORE_2_X +
MODULES_PIPELINE_CONNECTORS +
MODULES_MYSQL_SOURCE +
MODULES_MYSQL_PIPELINE +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,8 @@
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.SupportsPreCommitTopology;
import org.apache.flink.streaming.api.connector.sink2.SupportsPreWriteTopology;
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;
Expand Down Expand Up @@ -128,6 +130,8 @@ void sinkTo(
// Pre-write topology
if (sink instanceof WithPreWriteTopology) {
stream = ((WithPreWriteTopology<Event>) sink).addPreWriteTopology(stream);
} else if (sink instanceof SupportsPreWriteTopology) {
stream = ((SupportsPreWriteTopology<Event>) sink).addPreWriteTopology(stream);
}

if (sink instanceof TwoPhaseCommittingSink) {
Expand Down Expand Up @@ -192,6 +196,8 @@ private <CommT> void addCommittingTopology(
if (sink instanceof WithPreCommitTopology) {
preCommitted =
((WithPreCommitTopology<Event, CommT>) sink).addPreCommitTopology(written);
} else if (sink instanceof SupportsPreCommitTopology) {
preCommitted = ((SupportsPreCommitTopology) sink).addPreCommitTopology(written);
}

// TODO: Hard coding checkpoint
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,10 @@ private void sendFromTables(Consumer<TableId> tableIdConsumer) {
}

@Override
public void run(SourceContext<Event> context) throws InterruptedException {
public void run(
org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext<Event>
context)
throws InterruptedException {
Schema initialSchema =
Schema.newBuilder()
.physicalColumn("id", DataTypes.STRING())
Expand Down Expand Up @@ -265,7 +268,10 @@ private BinaryRecordData generateBinRec(Schema schema) {
return generator.generate(rowObjects);
}

private void collect(SourceContext<Event> sourceContext, Event event) {
private void collect(
org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext<Event>
sourceContext,
Event event) {
LOG.info("{}> Emitting event {}", subTaskId, event);
sourceContext.collect(event);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ limitations under the License.
<properties>
<doris.connector.version>25.1.0</doris.connector.version>
<mysql.connector.version>8.0.26</mysql.connector.version>
<doris.flink.major.version>1.20</doris.flink.major.version>
</properties>

<dependencies>
Expand All @@ -48,7 +49,7 @@ limitations under the License.

<dependency>
<groupId>org.apache.doris</groupId>
<artifactId>flink-doris-connector-${flink.major.version}</artifactId>
<artifactId>flink-doris-connector-${doris.flink.major.version}</artifactId>
<version>${doris.connector.version}</version>
</dependency>

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,135 @@
/*
* 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.doris.flink.sink.batch;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.connector.sink2.InitContextAdapter;
import org.apache.flink.api.connector.sink2.Sink;
import org.apache.flink.api.connector.sink2.SinkWriter;
import org.apache.flink.api.connector.sink2.WriterInitContext;
import org.apache.flink.util.Preconditions;

import org.apache.doris.flink.cfg.DorisExecutionOptions;
import org.apache.doris.flink.cfg.DorisOptions;
import org.apache.doris.flink.cfg.DorisReadOptions;
import org.apache.doris.flink.sink.writer.serializer.DorisRecordSerializer;

import java.io.IOException;

/**
* Copy from <a
* href="https://github.com/apache/doris-flink-connector/blob/25.1.0/flink-doris-connector/src/main/java/org/apache/doris/flink/sink/batch/DorisBatchSink.java">DorisBatchSink</a>
* to add {@link #createWriter(WriterInitContext)}} method.
*/
@Deprecated
@PublicEvolving
public class DorisBatchSink<IN> implements Sink<IN> {
private final DorisOptions dorisOptions;
private final DorisReadOptions dorisReadOptions;
private final DorisExecutionOptions dorisExecutionOptions;
private final DorisRecordSerializer<IN> serializer;

public DorisBatchSink(
DorisOptions dorisOptions,
DorisReadOptions dorisReadOptions,
DorisExecutionOptions dorisExecutionOptions,
DorisRecordSerializer<IN> serializer) {
this.dorisOptions = dorisOptions;
this.dorisReadOptions = dorisReadOptions;
this.dorisExecutionOptions = dorisExecutionOptions;
this.serializer = serializer;
}

@Override
public SinkWriter<IN> createWriter(InitContext initContext) throws IOException {
DorisBatchWriter<IN> dorisBatchWriter =
new DorisBatchWriter<IN>(
initContext,
serializer,
dorisOptions,
dorisReadOptions,
dorisExecutionOptions);
return dorisBatchWriter;
}

@Override
public SinkWriter<IN> createWriter(WriterInitContext context) throws IOException {
DorisBatchWriter<IN> dorisBatchWriter =
new DorisBatchWriter<IN>(
new InitContextAdapter(context),
serializer,
dorisOptions,
dorisReadOptions,
dorisExecutionOptions);
return dorisBatchWriter;
}

public static <IN> DorisBatchSink.Builder<IN> builder() {
return new DorisBatchSink.Builder<>();
}

/**
* build for DorisBatchSink.
*
* @param <IN> record type.
*/
public static class Builder<IN> {
private DorisOptions dorisOptions;
private DorisReadOptions dorisReadOptions;
private DorisExecutionOptions dorisExecutionOptions;
private DorisRecordSerializer<IN> serializer;

public DorisBatchSink.Builder<IN> setDorisOptions(DorisOptions dorisOptions) {
this.dorisOptions = dorisOptions;
return this;
}

public DorisBatchSink.Builder<IN> setDorisReadOptions(DorisReadOptions dorisReadOptions) {
this.dorisReadOptions = dorisReadOptions;
return this;
}

public DorisBatchSink.Builder<IN> setDorisExecutionOptions(
DorisExecutionOptions dorisExecutionOptions) {
this.dorisExecutionOptions = dorisExecutionOptions;
return this;
}

public DorisBatchSink.Builder<IN> setSerializer(DorisRecordSerializer<IN> serializer) {
this.serializer = serializer;
return this;
}

public DorisBatchSink<IN> build() {
Preconditions.checkNotNull(dorisOptions);
Preconditions.checkNotNull(dorisExecutionOptions);
Preconditions.checkNotNull(serializer);
if (dorisReadOptions == null) {
dorisReadOptions = DorisReadOptions.builder().build();
}
return new DorisBatchSink<>(
dorisOptions, dorisReadOptions, dorisExecutionOptions, serializer);
}
}

@VisibleForTesting
public DorisReadOptions getDorisReadOptions() {
return dorisReadOptions;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,6 @@

package org.apache.flink.cdc.connectors.doris.sink;

import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.cdc.common.configuration.Configuration;
import org.apache.flink.cdc.common.data.binary.BinaryRecordData;
import org.apache.flink.cdc.common.data.binary.BinaryStringData;
Expand Down Expand Up @@ -48,9 +46,11 @@
import org.apache.flink.cdc.connectors.doris.sink.utils.DorisSinkTestBase;
import org.apache.flink.cdc.connectors.doris.utils.DorisSchemaUtils;
import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo;
import org.apache.flink.runtime.client.JobExecutionException;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.util.RestartStrategyUtils;

import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.AfterEach;
Expand Down Expand Up @@ -85,7 +85,7 @@ class DorisMetadataApplierITCase extends DorisSinkTestBase {
public static void before() {
env.setParallelism(DEFAULT_PARALLELISM);
env.enableCheckpointing(3000);
env.setRestartStrategy(RestartStrategies.noRestart());
RestartStrategyUtils.configureNoRestartStrategy(env);
}

@BeforeEach
Expand Down Expand Up @@ -631,8 +631,7 @@ void testMysqlDefaultTimestampValueConversionInAddColumn(boolean batchMode) thro
}

private void runJobWithEvents(List<Event> events, boolean batchMode) throws Exception {
DataStream<Event> stream =
env.fromCollection(events, TypeInformation.of(Event.class)).setParallelism(1);
DataStream<Event> stream = env.fromData(events, new EventTypeInfo()).setParallelism(1);

Configuration config =
new Configuration()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.flink.cdc.connectors.doris.sink;

import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.connector.sink2.Sink;
import org.apache.flink.cdc.common.configuration.Configuration;
Expand All @@ -37,6 +36,7 @@
import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.util.RestartStrategyUtils;

import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
Expand Down Expand Up @@ -69,7 +69,7 @@ class DorisPipelineITCase extends DorisSinkTestBase {
public static void before() {
env.setParallelism(DEFAULT_PARALLELISM);
env.enableCheckpointing(3000);
env.setRestartStrategy(RestartStrategies.noRestart());
RestartStrategyUtils.configureNoRestartStrategy(env);
}

@BeforeEach
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@

package org.apache.flink.cdc.connectors.elasticsearch.sink;

import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.connector.sink2.Sink;
import org.apache.flink.cdc.common.event.Event;
Expand All @@ -37,6 +36,7 @@
import org.apache.flink.elasticsearch6.shaded.org.elasticsearch.client.RestHighLevelClient;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.util.RestartStrategyUtils;

import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
Expand Down Expand Up @@ -188,7 +188,7 @@ private StreamExecutionEnvironment createStreamExecutionEnvironment() {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(1);
env.enableCheckpointing(3000);
env.setRestartStrategy(RestartStrategies.noRestart());
RestartStrategyUtils.configureNoRestartStrategy(env);
return env;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@

package org.apache.flink.cdc.connectors.elasticsearch.sink;

import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.connector.sink2.Sink;
import org.apache.flink.cdc.common.event.Event;
Expand All @@ -37,6 +36,7 @@
import org.apache.flink.elasticsearch7.shaded.org.elasticsearch.client.RestHighLevelClient;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.util.RestartStrategyUtils;

import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
Expand Down Expand Up @@ -193,7 +193,7 @@ private StreamExecutionEnvironment createStreamExecutionEnvironment() {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(1);
env.enableCheckpointing(3000);
env.setRestartStrategy(RestartStrategies.noRestart());
RestartStrategyUtils.configureNoRestartStrategy(env);
return env;
}

Expand Down
Loading
Loading