Skip to content

Commit cca5223

Browse files
committed
[FLINK-39252][sqlserver] Fix SQL Server pipeline tests for Flink 2
1 parent 46da29d commit cca5223

4 files changed

Lines changed: 30 additions & 18 deletions

File tree

flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerFullTypesITCase.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818
package org.apache.flink.cdc.connectors.sqlserver.source;
1919

2020
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
21-
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
2221
import org.apache.flink.cdc.common.data.DateData;
2322
import org.apache.flink.cdc.common.data.DecimalData;
2423
import org.apache.flink.cdc.common.data.LocalZonedTimestampData;
@@ -37,6 +36,7 @@
3736
import org.apache.flink.cdc.connectors.sqlserver.source.config.SqlServerSourceConfigFactory;
3837
import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo;
3938
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
39+
import org.apache.flink.streaming.util.RestartStrategyUtils;
4040
import org.apache.flink.table.planner.factories.TestValuesTableFactory;
4141
import org.apache.flink.util.CloseableIterator;
4242

@@ -71,7 +71,7 @@ public void before() {
7171
TestValuesTableFactory.clearAllData();
7272
env.setParallelism(4);
7373
env.enableCheckpointing(200);
74-
env.setRestartStrategy(RestartStrategies.noRestart());
74+
RestartStrategyUtils.configureNoRestartStrategy(env);
7575
initializeSqlServerTable(DATABASE_NAME);
7676
}
7777

flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerOnlineSchemaMigrationITCase.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818
package org.apache.flink.cdc.connectors.sqlserver.source;
1919

2020
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
21-
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
2221
import org.apache.flink.cdc.common.data.binary.BinaryRecordData;
2322
import org.apache.flink.cdc.common.data.binary.BinaryStringData;
2423
import org.apache.flink.cdc.common.event.AddColumnEvent;
@@ -40,6 +39,7 @@
4039
import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
4140
import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo;
4241
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
42+
import org.apache.flink.streaming.util.RestartStrategyUtils;
4343
import org.apache.flink.table.planner.factories.TestValuesTableFactory;
4444
import org.apache.flink.util.CloseableIterator;
4545

@@ -73,7 +73,7 @@ void before() {
7373
TestValuesTableFactory.clearAllData();
7474
env.setParallelism(4);
7575
env.enableCheckpointing(200);
76-
env.setRestartStrategy(RestartStrategies.noRestart());
76+
RestartStrategyUtils.configureNoRestartStrategy(env);
7777
}
7878

7979
@Test

flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerPipelineITCase.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818
package org.apache.flink.cdc.connectors.sqlserver.source;
1919

2020
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
21-
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
2221
import org.apache.flink.cdc.common.configuration.Configuration;
2322
import org.apache.flink.cdc.common.data.binary.BinaryStringData;
2423
import org.apache.flink.cdc.common.event.CreateTableEvent;
@@ -40,6 +39,7 @@
4039
import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
4140
import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo;
4241
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
42+
import org.apache.flink.streaming.util.RestartStrategyUtils;
4343
import org.apache.flink.table.planner.factories.TestValuesTableFactory;
4444
import org.apache.flink.util.CloseableIterator;
4545

@@ -74,7 +74,7 @@ public void before() {
7474
TestValuesTableFactory.clearAllData();
7575
env.setParallelism(4);
7676
env.enableCheckpointing(200);
77-
env.setRestartStrategy(RestartStrategies.noRestart());
77+
RestartStrategyUtils.configureNoRestartStrategy(env);
7878
initializeSqlServerTable(DATABASE_NAME);
7979
}
8080

flink-cdc-connect/flink-cdc-pipeline-connectors/flink-cdc-pipeline-connector-sqlserver/src/test/java/org/apache/flink/cdc/connectors/sqlserver/source/SqlServerPipelineNewlyAddedTableITCase.java

Lines changed: 24 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818
package org.apache.flink.cdc.connectors.sqlserver.source;
1919

2020
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
21-
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
2221
import org.apache.flink.api.common.typeutils.TypeSerializer;
2322
import org.apache.flink.cdc.common.data.binary.BinaryStringData;
2423
import org.apache.flink.cdc.common.event.CreateTableEvent;
@@ -39,17 +38,20 @@
3938
import org.apache.flink.cdc.runtime.typeutils.BinaryRecordDataGenerator;
4039
import org.apache.flink.cdc.runtime.typeutils.EventTypeInfo;
4140
import org.apache.flink.configuration.Configuration;
41+
import org.apache.flink.configuration.StateRecoveryOptions;
4242
import org.apache.flink.core.execution.JobClient;
43+
import org.apache.flink.core.execution.SavepointFormatType;
4344
import org.apache.flink.runtime.checkpoint.CheckpointException;
44-
import org.apache.flink.runtime.jobgraph.SavepointConfigOptions;
4545
import org.apache.flink.streaming.api.datastream.DataStreamSource;
4646
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
4747
import org.apache.flink.streaming.api.operators.collect.AbstractCollectResultBuffer;
4848
import org.apache.flink.streaming.api.operators.collect.CheckpointedCollectResultBuffer;
4949
import org.apache.flink.streaming.api.operators.collect.CollectResultIterator;
50+
import org.apache.flink.streaming.api.operators.collect.CollectResultIteratorAdapter;
5051
import org.apache.flink.streaming.api.operators.collect.CollectSinkOperator;
5152
import org.apache.flink.streaming.api.operators.collect.CollectSinkOperatorFactory;
5253
import org.apache.flink.streaming.api.operators.collect.CollectStreamSink;
54+
import org.apache.flink.streaming.util.RestartStrategyUtils;
5355
import org.apache.flink.table.planner.factories.TestValuesTableFactory;
5456
import org.apache.flink.util.ExceptionUtils;
5557

@@ -245,7 +247,9 @@ void testAddNewTableWithExclude() throws Exception {
245247
new EventTypeInfo());
246248

247249
TypeSerializer<Event> serializer =
248-
source.getTransformation().getOutputType().createSerializer(env.getConfig());
250+
source.getTransformation()
251+
.getOutputType()
252+
.createSerializer(env.getConfig().getSerializerConfig());
249253
CheckpointedCollectResultBuffer<Event> resultBuffer =
250254
new CheckpointedCollectResultBuffer<>(serializer);
251255
String accumulatorName = "dataStreamCollect_" + UUID.randomUUID();
@@ -296,7 +300,9 @@ private void testAddNewTable(TestParam testParam, int parallelism) throws Except
296300
new EventTypeInfo());
297301

298302
TypeSerializer<Event> serializer =
299-
source.getTransformation().getOutputType().createSerializer(env.getConfig());
303+
source.getTransformation()
304+
.getOutputType()
305+
.createSerializer(env.getConfig().getSerializerConfig());
300306
CheckpointedCollectResultBuffer<Event> resultBuffer =
301307
new CheckpointedCollectResultBuffer<>(serializer);
302308
String accumulatorName = "dataStreamCollect_" + UUID.randomUUID();
@@ -445,7 +451,9 @@ private String triggerSavepointWithRetry(JobClient jobClient, String savepointDi
445451
// retry 600 times, it takes 100 milliseconds per time, at most retry 1 minute
446452
while (retryTimes < 600) {
447453
try {
448-
return jobClient.triggerSavepoint(savepointDirectory).get();
454+
return jobClient
455+
.triggerSavepoint(savepointDirectory, SavepointFormatType.DEFAULT)
456+
.get();
449457
} catch (Exception e) {
450458
Optional<CheckpointException> exception =
451459
ExceptionUtils.findThrowable(e, CheckpointException.class);
@@ -550,13 +558,17 @@ private <T> CollectResultIterator<T> addCollector(
550558
String accumulatorName) {
551559
CollectSinkOperatorFactory<T> sinkFactory =
552560
new CollectSinkOperatorFactory<>(serializer, accumulatorName);
553-
CollectSinkOperator<T> operator = (CollectSinkOperator<T>) sinkFactory.getOperator();
554-
CollectResultIterator<T> iterator =
555-
new CollectResultIterator<>(
556-
buffer, operator.getOperatorIdFuture(), accumulatorName, 0);
557561
CollectStreamSink<T> sink = new CollectStreamSink<>(source, sinkFactory);
558-
sink.name("Data stream collect sink");
562+
// Set both name and uid to the same value. The uid is used by Flink to generate
563+
// OperatorID via StreamGraphHasherV2.generateUserSpecifiedHash(uid), and the same
564+
// uid string must be passed to CollectResultIteratorAdapter for coordinator lookup.
565+
String operatorUid = "Data stream collect sink";
566+
sink.name(operatorUid).uid(operatorUid);
567+
CollectSinkOperator<T> operator = (CollectSinkOperator<T>) sinkFactory.getOperator();
559568
env.addOperator(sink.getTransformation());
569+
CollectResultIterator<T> iterator =
570+
new CollectResultIteratorAdapter<>(
571+
buffer, operatorUid, operator, accumulatorName, 0);
560572
env.registerCollectIterator(iterator);
561573
return iterator;
562574
}
@@ -565,13 +577,13 @@ private StreamExecutionEnvironment getStreamExecutionEnvironment(
565577
String finishedSavePointPath, int parallelism) {
566578
Configuration configuration = new Configuration();
567579
if (finishedSavePointPath != null) {
568-
configuration.setString(SavepointConfigOptions.SAVEPOINT_PATH, finishedSavePointPath);
580+
configuration.set(StateRecoveryOptions.SAVEPOINT_PATH, finishedSavePointPath);
569581
}
570582
StreamExecutionEnvironment env =
571583
StreamExecutionEnvironment.getExecutionEnvironment(configuration);
572584
env.setParallelism(parallelism);
573585
env.enableCheckpointing(500L);
574-
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 1000L));
586+
RestartStrategyUtils.configureFixedDelayRestartStrategy(env, 3, 1000L);
575587
return env;
576588
}
577589

0 commit comments

Comments
 (0)