Skip to content

Commit

Permalink
[FLINK-36249][streaming-java] Remove RestartStrategy-related configur…
Browse files Browse the repository at this point in the history
…ation getters/setters that return/set complex Java objects.

This closes apache#25256.
  • Loading branch information
JunRuiLee authored and zhuzhurk committed Sep 18, 2024
1 parent 06d045f commit 4e82510
Show file tree
Hide file tree
Showing 117 changed files with 764 additions and 1,657 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,19 +17,17 @@

package org.apache.flink.connector.base.sink;

import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.runtime.client.JobExecutionException;
import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.util.RestartStrategyUtils;
import org.apache.flink.test.junit5.MiniClusterExtension;
import org.apache.flink.util.TestLoggerExtension;

import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.junit.jupiter.api.extension.RegisterExtension;

import java.time.Duration;

import static org.assertj.core.api.Assertions.assertThatThrownBy;

/** Integration tests of a baseline generic sink that implements the AsyncSinkBase. */
Expand Down Expand Up @@ -67,7 +65,7 @@ public void testFailuresOnPersistingToDestinationAreCaughtAndRaised() {
@Test
public void testThatNoIssuesOccurWhenCheckpointingIsEnabled() throws Exception {
env.enableCheckpointing(20);
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, Duration.ofMillis(200)));
RestartStrategyUtils.configureFixedDelayRestartStrategy(env, 1, 200);
env.fromSequence(1, 10_000).map(Object::toString).sinkTo(new ArrayListAsyncSink());
env.execute("Integration Test: AsyncSinkBaseITCase");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@

import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.connector.source.Boundedness;
import org.apache.flink.api.connector.source.Source;
import org.apache.flink.connector.base.source.reader.mocks.MockBaseSource;
Expand All @@ -32,6 +31,7 @@
import org.apache.flink.streaming.api.datastream.DataStreamUtils;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.operators.collect.ClientAndIterator;
import org.apache.flink.streaming.util.RestartStrategyUtils;
import org.apache.flink.test.util.MiniClusterWithClientResource;
import org.apache.flink.util.TestLogger;

Expand Down Expand Up @@ -122,10 +122,11 @@ private void testHybridSource(FailoverType failoverType, Source source) throws E

final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(PARALLELISM);
env.setRestartStrategy(
FailoverType.NONE == failoverType
? RestartStrategies.noRestart()
: RestartStrategies.fixedDelayRestart(1, 0));
if (FailoverType.NONE == failoverType) {
RestartStrategyUtils.configureNoRestartStrategy(env);
} else {
RestartStrategyUtils.configureFixedDelayRestartStrategy(env, 1, 0);
}

final DataStream<Integer> stream =
env.fromSource(source, WatermarkStrategy.noWatermarks(), "hybrid-source")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import org.apache.flink.api.common.accumulators.ListAccumulator;
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.common.functions.OpenContext;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.connector.source.Boundedness;
import org.apache.flink.api.connector.source.Source;
import org.apache.flink.api.connector.source.SplitEnumerator;
Expand All @@ -32,6 +31,7 @@
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
import org.apache.flink.streaming.util.RestartStrategyUtils;
import org.apache.flink.test.util.AbstractTestBaseJUnit4;
import org.apache.flink.util.FlinkRuntimeException;

Expand Down Expand Up @@ -74,7 +74,7 @@ public void testEnumeratorCreationFails() throws Exception {
OnceFailingToCreateEnumeratorSource.reset();

final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 0L));
RestartStrategyUtils.configureFixedDelayRestartStrategy(env, Integer.MAX_VALUE, 0L);
final Source<Integer, ?, ?> source =
new OnceFailingToCreateEnumeratorSource(2, 10, Boundedness.BOUNDED);
final DataStream<Integer> stream =
Expand All @@ -87,7 +87,7 @@ public void testEnumeratorRestoreFails() throws Exception {
OnceFailingToRestoreEnumeratorSource.reset();

final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 0L));
RestartStrategyUtils.configureFixedDelayRestartStrategy(env, Integer.MAX_VALUE, 0L);
env.enableCheckpointing(10);

final Source<Integer, ?, ?> source =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
package org.apache.flink.connector.base.source.reader;

import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.state.CheckpointListener;
import org.apache.flink.configuration.CheckpointingOptions;
import org.apache.flink.configuration.Configuration;
Expand All @@ -30,6 +29,7 @@
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.util.RestartStrategyUtils;
import org.apache.flink.test.util.MiniClusterWithClientResource;
import org.apache.flink.util.TestLogger;

Expand Down Expand Up @@ -124,7 +124,7 @@ private StreamExecutionEnvironment createEnv(
env.getCheckpointConfig()
.setExternalizedCheckpointRetention(
ExternalizedCheckpointRetention.RETAIN_ON_CANCELLATION);
env.setRestartStrategy(RestartStrategies.noRestart());
RestartStrategyUtils.configureNoRestartStrategy(env);

DataStream<Long> stream = env.fromSequence(0, Long.MAX_VALUE);
stream.map(new FailingMapFunction(restoreCheckpoint == null)).addSink(new SleepySink());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@

import org.apache.flink.api.common.RuntimeExecutionMode;
import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.connector.source.Boundedness;
import org.apache.flink.configuration.Configuration;
Expand All @@ -32,8 +31,7 @@
import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
import org.apache.flink.streaming.api.graph.StreamGraph;
import org.apache.flink.streaming.api.operators.StreamSource;

import java.time.Duration;
import org.apache.flink.streaming.util.RestartStrategyUtils;

/** Tests the functionality of the {@link FileSink} in BATCH mode. */
class BatchExecutionFileSinkITCase extends FileSinkITBase {
Expand All @@ -50,9 +48,9 @@ protected JobGraph createJobGraph(boolean triggerFailover, String path) {
env.configure(config, getClass().getClassLoader());

if (triggerFailover) {
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, Duration.ofMillis(100)));
RestartStrategyUtils.configureFixedDelayRestartStrategy(env, 1, 100);
} else {
env.setRestartStrategy(RestartStrategies.noRestart());
RestartStrategyUtils.configureNoRestartStrategy(env);
}

// Create a testing job with a bounded legacy source in a bit hacky way.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
package org.apache.flink.connector.file.sink;

import org.apache.flink.api.common.RuntimeExecutionMode;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.configuration.Configuration;
Expand Down Expand Up @@ -53,6 +52,7 @@
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
import org.apache.flink.streaming.api.graph.StreamGraph;
import org.apache.flink.streaming.util.RestartStrategyUtils;
import org.apache.flink.test.junit5.MiniClusterExtension;
import org.apache.flink.testutils.junit.SharedObjectsExtension;
import org.apache.flink.testutils.junit.SharedReference;
Expand Down Expand Up @@ -222,7 +222,7 @@ private JobGraph createJobGraph(
env.configure(config, getClass().getClassLoader());

env.enableCheckpointing(100, CheckpointingMode.EXACTLY_ONCE);
env.setRestartStrategy(RestartStrategies.noRestart());
RestartStrategyUtils.configureNoRestartStrategy(env);
env.getCheckpointConfig().setCheckpointStorage(new FileSystemCheckpointStorage(cpPath));
env.setStateBackend(new HashMapStateBackend());

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
package org.apache.flink.connector.file.sink;

import org.apache.flink.api.common.RuntimeExecutionMode;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.configuration.Configuration;
Expand All @@ -34,11 +33,11 @@
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
import org.apache.flink.streaming.api.graph.StreamGraph;
import org.apache.flink.streaming.util.RestartStrategyUtils;

import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;

import java.time.Duration;
import java.util.Collections;
import java.util.Map;
import java.util.UUID;
Expand Down Expand Up @@ -82,9 +81,9 @@ protected JobGraph createJobGraph(boolean triggerFailover, String path) {
env.enableCheckpointing(10, CheckpointingMode.EXACTLY_ONCE);

if (triggerFailover) {
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, Duration.ofMillis(100)));
RestartStrategyUtils.configureFixedDelayRestartStrategy(env, 1, 100);
} else {
env.setRestartStrategy(RestartStrategies.noRestart());
RestartStrategyUtils.configureNoRestartStrategy(env);
}

DataStreamSink<Integer> sink =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.RuntimeExecutionMode;
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.configuration.BatchExecutionOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.connector.file.src.reader.TextLineInputFormat;
Expand All @@ -36,6 +35,7 @@
import org.apache.flink.streaming.api.datastream.DataStreamUtils;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.operators.collect.ClientAndIterator;
import org.apache.flink.streaming.util.RestartStrategyUtils;
import org.apache.flink.test.junit5.InjectMiniCluster;
import org.apache.flink.test.junit5.MiniClusterExtension;
import org.apache.flink.test.util.MiniClusterWithClientResource;
Expand Down Expand Up @@ -148,7 +148,7 @@ private void testBoundedTextFileSource(
.build();

final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
RestartStrategyUtils.configureFixedDelayRestartStrategy(env, 1, 0L);
env.setParallelism(PARALLELISM);

if (batchMode) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,14 @@
package org.apache.flink.connectors.hive;

import org.apache.flink.api.common.eventtime.WatermarkStrategy;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
import org.apache.flink.connector.datagen.source.TestDataGenerators;
import org.apache.flink.connector.file.table.FileSystemConnectorOptions;
import org.apache.flink.core.fs.Path;
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.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.ExplainDetail;
import org.apache.flink.table.api.Expressions;
Expand Down Expand Up @@ -774,7 +774,7 @@ private void testStreamingWriteWithCustomPartitionCommitPolicy(
env.setParallelism(1);
env.enableCheckpointing(100);
// avoid the job to restart infinitely
env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 1_000));
RestartStrategyUtils.configureFixedDelayRestartStrategy(env, 3, 1000L);

StreamTableEnvironment tEnv = HiveTestUtils.createTableEnvInStreamingMode(env);
tEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,11 +44,9 @@ public class ArchivedExecutionConfig implements Serializable {

public ArchivedExecutionConfig(ExecutionConfig ec) {
executionMode = ec.getExecutionMode().name();
if (ec.getRestartStrategy() != null) {
restartStrategyDescription = ec.getRestartStrategy().getDescription();
} else {
restartStrategyDescription = "default";
}
restartStrategyDescription =
RestartStrategyDescriptionUtils.getRestartStrategyDescription(ec.toConfiguration());

maxParallelism = ec.getMaxParallelism();
parallelism = ec.getParallelism();
objectReuseEnabled = ec.isObjectReuseEnabled();
Expand Down
Loading

0 comments on commit 4e82510

Please sign in to comment.