From af5279e9bd3dec18512a54c59982e3dc1f253cd2 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 2 Apr 2018 11:13:57 +0200 Subject: [PATCH] [FLINK-9121] [flip6] Remove Flip6 prefixes and other references Remove Flip6 prefixes and references to make Flip-6 the proper default: Rename categories Flip6 -> New and OldAndFlip6 -> LegacyAndNew Remove Flip-6 from documentation Remove Flip-6 from start up scripts This closes #5801. --- .travis.yml | 10 ++-- docs/monitoring/rest_api.md | 4 -- docs/ops/config.md | 10 ++-- docs/ops/state/large_state_tuning.md | 4 +- .../apache/flink/client/LocalExecutor.java | 2 +- .../apache/flink/client/RemoteExecutor.java | 2 +- .../apache/flink/client/cli/CliFrontend.java | 12 ++-- .../apache/flink/client/cli/DefaultCLI.java | 9 ++- .../{Flip6DefaultCLI.java => LegacyCLI.java} | 21 +++---- ...=> LegacyStandaloneClusterDescriptor.java} | 33 ++++++----- .../StandaloneClusterDescriptor.java | 21 +++---- .../cli/CliFrontendPackageProgramTest.java | 2 +- .../flink/client/cli/CliFrontendTestBase.java | 10 ++-- .../flink/client/cli/DefaultCLITest.java | 2 +- .../program/rest/RestClusterClientTest.java | 4 +- .../flink/configuration/CoreOptions.java | 16 +++--- flink-dist/src/main/flink-bin/bin/config.sh | 2 +- .../src/main/flink-bin/bin/jobmanager.sh | 2 +- .../src/main/flink-bin/bin/start-cluster.sh | 2 +- .../src/main/flink-bin/bin/stop-cluster.sh | 10 ++-- .../src/main/flink-bin/bin/taskmanager.sh | 2 +- .../flink-bin/mesos-bin/mesos-appmaster.sh | 2 +- .../flink-bin/mesos-bin/mesos-taskmanager.sh | 2 +- flink-docs/pom.xml | 2 +- .../client/gateway/local/DependencyTest.java | 4 +- .../environment/PythonEnvironmentFactory.java | 3 +- .../client/proxy/KvStateClientProxyImpl.java | 2 +- .../proxy/KvStateClientProxyImplTest.java | 2 +- .../HighAvailabilityServices.java | 6 +- .../flink/runtime/instance/SharedSlot.java | 2 +- .../flink/runtime/instance/SimpleSlot.java | 2 +- .../apache/flink/runtime/instance/Slot.java | 4 +- .../scheduler/CoLocationConstraint.java | 3 +- .../runtime/minicluster/MiniCluster.java | 6 +- .../flink/runtime/query/KvStateRegistry.java | 2 +- .../runtime/taskmanager/TaskManager.scala | 2 +- .../runtime/dispatcher/DispatcherTest.java | 4 +- .../FileArchivedExecutionGraphStoreTest.java | 4 +- .../dispatcher/MiniDispatcherTest.java | 4 +- .../heartbeat/HeartbeatManagerTest.java | 4 +- .../standalone/StandaloneHaServicesTest.java | 2 +- .../scheduler/SchedulerSlotSharingTest.java | 2 +- .../runtime/jobmaster/JobMasterTest.java | 4 +- .../runtime/jobmaster/JobResultTest.java | 4 +- .../slotpool/AllocatedSlotsTest.java | 4 +- .../slotpool/AvailableSlotsTest.java | 4 +- .../jobmaster/slotpool/DualKeyMapTest.java | 4 +- .../jobmaster/slotpool/SlotPoolRpcTest.java | 4 +- .../slotpool/SlotPoolSchedulingTestBase.java | 4 +- .../jobmaster/slotpool/SlotPoolTest.java | 4 +- .../slotpool/SlotSharingManagerTest.java | 4 +- .../minicluster/MiniClusterITCase.java | 4 +- .../runtime/query/KvStateRegistryTest.java | 2 +- .../JobLeaderIdServiceTest.java | 4 +- .../ResourceManagerHATest.java | 4 +- .../ResourceManagerJobMasterTest.java | 4 +- .../ResourceManagerTaskExecutorTest.java | 4 +- .../resourcemanager/ResourceManagerTest.java | 4 +- .../slotmanager/SlotManagerTest.java | 4 +- .../slotmanager/SlotProtocolTest.java | 4 +- .../rest/RestServerEndpointITCase.java | 4 +- .../runtime/rest/RestServerEndpointTest.java | 4 +- .../job/BlobServerPortHandlerTest.java | 4 +- .../job/JobExecutionResultHandlerTest.java | 4 +- .../handler/job/JobSubmitHandlerTest.java | 4 +- .../metrics/AbstractMetricsHandlerTest.java | 4 +- .../job/metrics/MetricsHandlerTestBase.java | 4 +- .../rest/messages/MessageParametersTest.java | 4 +- .../RestRequestMarshallingTestBase.java | 4 +- .../RestResponseMarshallingTestBase.java | 4 +- .../metrics/AbstractMetricsHeadersTest.java | 4 +- .../metrics/JobManagerMetricsHeadersTest.java | 4 +- .../job/metrics/JobMetricsHeadersTest.java | 4 +- .../metrics/JobVertexMetricsHeadersTest.java | 4 +- .../metrics/MetricsFilterParameterTest.java | 4 +- .../metrics/SubtaskMetricsHeadersTest.java | 4 +- .../TaskManagerMetricsHeadersTest.java | 4 +- .../TaskManagerIdPathParameterTest.java | 4 +- .../flink/runtime/rpc/AsyncCallsTest.java | 4 +- .../runtime/rpc/FencedRpcEndpointTest.java | 4 +- .../flink/runtime/rpc/RpcConnectionTest.java | 4 +- .../flink/runtime/rpc/RpcEndpointTest.java | 4 +- .../runtime/rpc/akka/AkkaRpcActorTest.java | 4 +- .../runtime/rpc/akka/AkkaRpcServiceTest.java | 4 +- .../rpc/akka/MainThreadValidationTest.java | 4 +- .../rpc/akka/MessageSerializationTest.java | 4 +- .../NetworkBufferCalculationTest.java | 4 +- .../taskexecutor/TaskExecutorITCase.java | 4 +- .../taskexecutor/TaskExecutorTest.java | 4 +- .../TaskManagerServicesConfigurationTest.java | 4 +- .../taskexecutor/TaskManagerServicesTest.java | 4 +- .../TestingTaskExecutorGateway.java | 4 +- .../taskexecutor/slot/TimerServiceTest.java | 4 +- ...TaskCancelAsyncProducerConsumerITCase.java | 4 +- .../flink/api/scala/ScalaShellITCase.scala | 2 +- .../scala/ScalaShellLocalStartupITCase.scala | 2 +- ...java => LegacyLocalStreamEnvironment.java} | 54 +++++++----------- .../environment/LocalStreamEnvironment.java | 57 +++++++++++-------- .../environment/RemoteStreamEnvironment.java | 2 +- .../StreamExecutionEnvironment.java | 6 +- .../LocalStreamEnvironmentITCase.java | 6 +- .../{Flip6.java => LegacyAndNew.java} | 5 +- .../category/{OldAndFlip6.java => New.java} | 4 +- .../flink/test/util/AbstractTestBase.java | 4 +- .../flink/test/util/MiniClusterResource.java | 22 +++---- .../JobMasterTriggerSavepointIT.java | 4 +- .../accumulators/AccumulatorLiveITCase.java | 4 +- .../test/misc/AutoParallelismITCase.java | 2 +- .../operators/RemoteEnvironmentITCase.java | 4 +- ...erHAProcessFailureBatchRecoveryITCase.java | 2 +- .../ProcessFailureCancelingITCase.java | 2 +- ...agerProcessFailureBatchRecoveryITCase.java | 2 +- ...ProcessFailureStreamingRecoveryITCase.java | 2 +- .../AbstractOperatorRestoreTestBase.java | 4 +- .../yarn/TestingYarnClusterDescriptor.java | 2 +- .../yarn/YARNHighAvailabilityITCase.java | 2 +- .../org/apache/flink/yarn/YARNITCase.java | 8 +-- .../YARNSessionCapacitySchedulerITCase.java | 4 +- .../flink/yarn/YARNSessionFIFOITCase.java | 4 +- .../flink/yarn/YarnConfigurationITCase.java | 2 +- .../org/apache/flink/yarn/YarnTestBase.java | 8 +-- .../yarn/AbstractYarnClusterDescriptor.java | 2 +- ....java => LegacyYarnClusterDescriptor.java} | 55 ++++++------------ .../flink/yarn/YarnClusterDescriptor.java | 50 +++++++++++----- .../flink/yarn/cli/FlinkYarnSessionCli.java | 14 ++--- .../flink/yarn/YarnClusterDescriptorTest.java | 16 +++--- pom.xml | 12 ++-- 127 files changed, 409 insertions(+), 410 deletions(-) rename flink-clients/src/main/java/org/apache/flink/client/cli/{Flip6DefaultCLI.java => LegacyCLI.java} (75%) rename flink-clients/src/main/java/org/apache/flink/client/deployment/{Flip6StandaloneClusterDescriptor.java => LegacyStandaloneClusterDescriptor.java} (59%) rename flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/{Flip6LocalStreamEnvironment.java => LegacyLocalStreamEnvironment.java} (64%) rename flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/category/{Flip6.java => LegacyAndNew.java} (87%) rename flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/category/{OldAndFlip6.java => New.java} (90%) rename flink-yarn/src/main/java/org/apache/flink/yarn/{Flip6YarnClusterDescriptor.java => LegacyYarnClusterDescriptor.java} (55%) diff --git a/.travis.yml b/.travis.yml index f86bbef1638c0..08aab30331677 100644 --- a/.travis.yml +++ b/.travis.yml @@ -39,27 +39,27 @@ matrix: - jdk: "oraclejdk8" env: - TEST="core" - - PROFILE="-Dhadoop.version=2.8.3 -Dflip6" + - PROFILE="-Dhadoop.version=2.8.3 -Dnew" - CACHE_NAME=JDK8_H280_CO - jdk: "oraclejdk8" env: - TEST="libraries" - - PROFILE="-Dhadoop.version=2.8.3 -Dflip6" + - PROFILE="-Dhadoop.version=2.8.3 -Dnew" - CACHE_NAME=JDK8_H280_L - jdk: "oraclejdk8" env: - TEST="connectors" - - PROFILE="-Dhadoop.version=2.8.3 -Dflip6 -Pinclude-kinesis" + - PROFILE="-Dhadoop.version=2.8.3 -Dnew -Pinclude-kinesis" - CACHE_NAME=JDK8_H280_CN - jdk: "oraclejdk8" env: - TEST="tests" - - PROFILE="-Dhadoop.version=2.8.3 -Dflip6" + - PROFILE="-Dhadoop.version=2.8.3 -Dnew" - CACHE_NAME=JDK8_H280_T - jdk: "oraclejdk8" env: - TEST="misc" - - PROFILE="-Dhadoop.version=2.8.3 -Dflip6 -Dinclude_hadoop_aws" + - PROFILE="-Dhadoop.version=2.8.3 -Dnew -Dinclude_hadoop_aws" - CACHE_NAME=JDK8_H280_M - jdk: "openjdk8" env: diff --git a/docs/monitoring/rest_api.md b/docs/monitoring/rest_api.md index 410ba9cacbf4f..c160ad08ff465 100644 --- a/docs/monitoring/rest_api.md +++ b/docs/monitoring/rest_api.md @@ -674,10 +674,6 @@ Response: {"jobid": "869a9868d49c679e7355700e0857af85"} ~~~ -## FLIP-6 - -The following is the REST API documentation for FLIP-6. - ### Dispatcher {% include generated/rest_dispatcher.html %} diff --git a/docs/ops/config.md b/docs/ops/config.md index ce5bc9b981bee..d5d81f505d56a 100644 --- a/docs/ops/config.md +++ b/docs/ops/config.md @@ -378,15 +378,15 @@ You have to configure `jobmanager.archive.fs.dir` in order to archive terminated {% include generated/history_server_configuration.html %} -## Flip-6 +### Slot Manager -- `mode`: Execution mode of Flink. Possible values are `old` and `flip6`. In order to start the Flip-6 components, you have to specify `flip6` (DEFAULT: `old`). +The configuration keys in this section are relevant for the SlotManager running in the ResourceManager -### Slot Manager (Flip-6) +{% include generated/slot_manager_configuration.html %} -The configuration keys in this section are relevant for the SlotManager running in the Flip-6 ResourceManager +## Legacy -{% include generated/slot_manager_configuration.html %} +- `mode`: Execution mode of Flink. Possible values are `legacy` and `new`. In order to start the legacy components, you have to specify `legacy` (DEFAULT: `new`). ## Background diff --git a/docs/ops/state/large_state_tuning.md b/docs/ops/state/large_state_tuning.md index d6cec6f6e3217..635993849f745 100644 --- a/docs/ops/state/large_state_tuning.md +++ b/docs/ops/state/large_state_tuning.md @@ -321,10 +321,12 @@ files also no additional disk space is consumed for task-local recovery with inc ### Allocation-preserving scheduling -Task-local recovery assumes allocation-preserving task scheduling under failures, which was introduced as part of FLIP-6 and works as follows. Each task remembers its previous +Task-local recovery assumes allocation-preserving task scheduling under failures, which works as follows. Each task remembers its previous allocation and *requests the exact same slot* to restart in recovery. If this slot is not available, the task will request a *new, fresh slot* from the resource manager. This way, if a task manager is no longer available, a task that cannot return to its previous location *will not drive other recovering tasks out of their previous slots*. Our reasoning is that the previous slot can only disappear when a task manager is no longer available, and in this case *some* tasks have to request a new slot anyways. With our scheduling strategy we give the maximum number of tasks a chance to recover from their local state and avoid the cascading effect of tasks stealing their previous slots from one another. +Allocation-preserving scheduling does not work with Flink's legacy mode. + {% top %} diff --git a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java index 415e33cfbcbf5..48855be3188e5 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/LocalExecutor.java @@ -124,7 +124,7 @@ public void start() throws Exception { private JobExecutorService createJobExecutorService(Configuration configuration) throws Exception { final JobExecutorService newJobExecutorService; - if (CoreOptions.FLIP6_MODE.equals(configuration.getString(CoreOptions.MODE))) { + if (CoreOptions.NEW_MODE.equals(configuration.getString(CoreOptions.MODE))) { configuration.setInteger(RestOptions.REST_PORT, 0); diff --git a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java index 1afa087688820..f6242e797a6da 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/RemoteExecutor.java @@ -151,7 +151,7 @@ public int getDefaultParallelism() { public void start() throws Exception { synchronized (lock) { if (client == null) { - if (CoreOptions.OLD_MODE.equals(clientConfiguration.getString(CoreOptions.MODE))) { + if (CoreOptions.LEGACY_MODE.equals(clientConfiguration.getString(CoreOptions.MODE))) { client = new StandaloneClusterClient(clientConfiguration); } else { client = new RestClusterClient<>(clientConfiguration, "RemoteExecutor"); diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java index a874891eb25c7..ce6556ba8c9bc 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontend.java @@ -119,7 +119,7 @@ public class CliFrontend { private final int defaultParallelism; - private final boolean flip6; + private final boolean isNewMode; public CliFrontend( Configuration configuration, @@ -144,7 +144,7 @@ public CliFrontend( this.clientTimeout = AkkaUtils.getClientTimeout(this.configuration); this.defaultParallelism = configuration.getInteger(CoreOptions.DEFAULT_PARALLELISM); - this.flip6 = CoreOptions.FLIP6_MODE.equalsIgnoreCase(configuration.getString(CoreOptions.MODE)); + this.isNewMode = CoreOptions.NEW_MODE.equalsIgnoreCase(configuration.getString(CoreOptions.MODE)); } // -------------------------------------------------------------------------------------------- @@ -225,7 +225,7 @@ private void runProgram( final ClusterClient client; // directly deploy the job if the cluster is started in job mode and detached - if (flip6 && clusterId == null && runOptions.getDetachedMode()) { + if (isNewMode && clusterId == null && runOptions.getDetachedMode()) { int parallelism = runOptions.getParallelism() == -1 ? defaultParallelism : runOptions.getParallelism(); final JobGraph jobGraph = PackagedProgramUtils.createJobGraph(program, configuration, parallelism); @@ -1162,10 +1162,10 @@ public static List> loadCustomCommandLines(Configuration co LOG.warn("Could not load CLI class {}.", flinkYarnSessionCLI, e); } - if (configuration.getString(CoreOptions.MODE).equalsIgnoreCase(CoreOptions.FLIP6_MODE)) { - customCommandLines.add(new Flip6DefaultCLI(configuration)); - } else { + if (configuration.getString(CoreOptions.MODE).equalsIgnoreCase(CoreOptions.NEW_MODE)) { customCommandLines.add(new DefaultCLI(configuration)); + } else { + customCommandLines.add(new LegacyCLI(configuration)); } return customCommandLines; diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java index 2ca5a65604bca..43efc63a85dd4 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/DefaultCLI.java @@ -25,6 +25,7 @@ import org.apache.flink.util.FlinkException; import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Options; import javax.annotation.Nullable; @@ -39,13 +40,17 @@ public DefaultCLI(Configuration configuration) { @Override public boolean isActive(CommandLine commandLine) { - // always active because we can try to read a JobManager address from the config return true; } @Override public String getId() { - return "Default CLI"; + return "default"; + } + + @Override + public void addGeneralOptions(Options baseOptions) { + super.addGeneralOptions(baseOptions); } @Override diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java b/flink-clients/src/main/java/org/apache/flink/client/cli/LegacyCLI.java similarity index 75% rename from flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java rename to flink-clients/src/main/java/org/apache/flink/client/cli/LegacyCLI.java index 3ff1b3e611519..761929dbacf5c 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/Flip6DefaultCLI.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/LegacyCLI.java @@ -19,46 +19,41 @@ package org.apache.flink.client.cli; import org.apache.flink.client.deployment.ClusterSpecification; -import org.apache.flink.client.deployment.Flip6StandaloneClusterDescriptor; +import org.apache.flink.client.deployment.LegacyStandaloneClusterDescriptor; import org.apache.flink.client.deployment.StandaloneClusterId; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.FlinkException; import org.apache.commons.cli.CommandLine; -import org.apache.commons.cli.Options; import javax.annotation.Nullable; /** - * The default CLI which is used for interaction with standalone clusters. + * The CLI which is used for interaction with the legacy standalone clusters. */ -public class Flip6DefaultCLI extends AbstractCustomCommandLine { +public class LegacyCLI extends AbstractCustomCommandLine { - public Flip6DefaultCLI(Configuration configuration) { + public LegacyCLI(Configuration configuration) { super(configuration); } @Override public boolean isActive(CommandLine commandLine) { + // always active because we can try to read a JobManager address from the config return true; } @Override public String getId() { - return "flip6"; + return "legacy"; } @Override - public void addGeneralOptions(Options baseOptions) { - super.addGeneralOptions(baseOptions); - } - - @Override - public Flip6StandaloneClusterDescriptor createClusterDescriptor( + public LegacyStandaloneClusterDescriptor createClusterDescriptor( CommandLine commandLine) throws FlinkException { final Configuration effectiveConfiguration = applyCommandLineOptionsToConfiguration(commandLine); - return new Flip6StandaloneClusterDescriptor(effectiveConfiguration); + return new LegacyStandaloneClusterDescriptor(effectiveConfiguration); } @Override diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/LegacyStandaloneClusterDescriptor.java similarity index 59% rename from flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java rename to flink-clients/src/main/java/org/apache/flink/client/deployment/LegacyStandaloneClusterDescriptor.java index 0a3286a3cc9b3..21e020ced557e 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/Flip6StandaloneClusterDescriptor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/LegacyStandaloneClusterDescriptor.java @@ -18,56 +18,55 @@ package org.apache.flink.client.deployment; -import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.client.program.StandaloneClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.util.FlinkException; -import org.apache.flink.util.Preconditions; /** * A deployment descriptor for an existing cluster. */ -public class Flip6StandaloneClusterDescriptor implements ClusterDescriptor { +public class LegacyStandaloneClusterDescriptor implements ClusterDescriptor { private final Configuration config; - public Flip6StandaloneClusterDescriptor(Configuration config) { - this.config = Preconditions.checkNotNull(config); + public LegacyStandaloneClusterDescriptor(Configuration config) { + this.config = config; } @Override public String getClusterDescription() { String host = config.getString(JobManagerOptions.ADDRESS, ""); int port = config.getInteger(JobManagerOptions.PORT, -1); - return "FLIP-6 Standalone cluster at " + host + ":" + port; + return "Standalone cluster at " + host + ":" + port; } @Override - public RestClusterClient retrieve(StandaloneClusterId standaloneClusterId) throws ClusterRetrieveException { + public StandaloneClusterClient retrieve(StandaloneClusterId standaloneClusterId) throws ClusterRetrieveException { try { - return new RestClusterClient<>(config, standaloneClusterId); + return new StandaloneClusterClient(config); } catch (Exception e) { - throw new ClusterRetrieveException("Couldn't retrieve FLIP-6 standalone cluster", e); + throw new ClusterRetrieveException("Couldn't retrieve standalone cluster", e); } } @Override - public RestClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) { - throw new UnsupportedOperationException("Can't deploy a FLIP-6 standalone cluster."); + public StandaloneClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) { + throw new UnsupportedOperationException("Can't deploy a standalone cluster."); } @Override - public RestClusterClient deployJobCluster( - ClusterSpecification clusterSpecification, - JobGraph jobGraph, - boolean detached) { - throw new UnsupportedOperationException("Can't deploy a standalone FLIP-6 per-job cluster."); + public StandaloneClusterClient deployJobCluster( + ClusterSpecification clusterSpecification, + JobGraph jobGraph, + boolean detached) { + throw new UnsupportedOperationException("Can't deploy a standalone per-job cluster."); } @Override public void terminateCluster(StandaloneClusterId clusterId) throws FlinkException { - throw new UnsupportedOperationException("Cannot terminate a Flip-6 standalone cluster."); + throw new UnsupportedOperationException("Cannot terminate standalone clusters."); } @Override diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java index 83bb13646f203..c4bcde690a64d 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/StandaloneClusterDescriptor.java @@ -18,11 +18,12 @@ package org.apache.flink.client.deployment; -import org.apache.flink.client.program.StandaloneClusterClient; +import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; /** * A deployment descriptor for an existing cluster. @@ -32,7 +33,7 @@ public class StandaloneClusterDescriptor implements ClusterDescriptor retrieve(StandaloneClusterId standaloneClusterId) throws ClusterRetrieveException { try { - return new StandaloneClusterClient(config); + return new RestClusterClient<>(config, standaloneClusterId); } catch (Exception e) { throw new ClusterRetrieveException("Couldn't retrieve standalone cluster", e); } } @Override - public StandaloneClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) { + public RestClusterClient deploySessionCluster(ClusterSpecification clusterSpecification) { throw new UnsupportedOperationException("Can't deploy a standalone cluster."); } @Override - public StandaloneClusterClient deployJobCluster( - ClusterSpecification clusterSpecification, - JobGraph jobGraph, - boolean detached) { + public RestClusterClient deployJobCluster( + ClusterSpecification clusterSpecification, + JobGraph jobGraph, + boolean detached) { throw new UnsupportedOperationException("Can't deploy a standalone per-job cluster."); } @Override public void terminateCluster(StandaloneClusterId clusterId) throws FlinkException { - throw new UnsupportedOperationException("Cannot terminate standalone clusters."); + throw new UnsupportedOperationException("Cannot terminate a standalone cluster."); } @Override diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java index 48c889120343c..e9c62f9028fc6 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java @@ -71,7 +71,7 @@ public void setup() throws Exception { final Configuration configuration = new Configuration(); frontend = new CliFrontend( configuration, - Collections.singletonList(new DefaultCLI(configuration))); + Collections.singletonList(new LegacyCLI(configuration))); } @Test diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestBase.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestBase.java index e2463f214c430..3c243763a0b43 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestBase.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendTestBase.java @@ -30,7 +30,7 @@ import java.util.List; /** - * Base test class for {@link CliFrontend} tests that wraps the Flip-6 vs. non-Flip-6 modes. + * Base test class for {@link CliFrontend} tests that wraps the new vs. legacy mode. */ @RunWith(Parameterized.class) public abstract class CliFrontendTestBase extends TestLogger { @@ -39,7 +39,7 @@ public abstract class CliFrontendTestBase extends TestLogger { @Parameterized.Parameters(name = "Mode = {0}") public static List parameters() { - return Arrays.asList(CoreOptions.OLD_MODE, CoreOptions.FLIP6_MODE); + return Arrays.asList(CoreOptions.LEGACY_MODE, CoreOptions.NEW_MODE); } protected Configuration getConfiguration() { @@ -51,10 +51,10 @@ protected Configuration getConfiguration() { static AbstractCustomCommandLine getCli(Configuration configuration) { switch (configuration.getString(CoreOptions.MODE)) { - case CoreOptions.OLD_MODE: + case CoreOptions.LEGACY_MODE: + return new LegacyCLI(configuration); + case CoreOptions.NEW_MODE: return new DefaultCLI(configuration); - case CoreOptions.FLIP6_MODE: - return new Flip6DefaultCLI(configuration); } throw new IllegalStateException(); } diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java index 8402b7e112099..6c13368c5fca6 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/DefaultCLITest.java @@ -34,7 +34,7 @@ import static org.junit.Assert.assertThat; /** - * Tests for the {@link DefaultCLI}. + * Tests for the {@link LegacyCLI}. */ public class DefaultCLITest extends CliFrontendTestBase { diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java index 4202de292f0ea..e7f9bf9c6c7d4 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/program/rest/RestClusterClientTest.java @@ -88,7 +88,7 @@ import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.util.ExecutorThreadFactory; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.OptionalFailure; @@ -148,7 +148,7 @@ *

These tests verify that the client uses the appropriate headers for each * request, properly constructs the request bodies/parameters and processes the responses correctly. */ -@Category(Flip6.class) +@Category(New.class) public class RestClusterClientTest extends TestLogger { private static final String REST_ADDRESS = "http://localhost:1234"; diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java index 8ac729d94f7f4..4f8c07404eda8 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CoreOptions.java @@ -274,20 +274,20 @@ public static ConfigOption fileSystemConnectionLimitStreamInactivityTimeou // ------------------------------------------------------------------------ /** - * Constant value for the Flip-6 execution mode. + * Constant value for the new execution mode. */ - public static final String FLIP6_MODE = "flip6"; + public static final String NEW_MODE = "new"; /** * Constant value for the old execution mode. */ - public static final String OLD_MODE = "old"; + public static final String LEGACY_MODE = "legacy"; /** - * Switch to select the execution mode. Possible values are 'flip6' and 'old'. + * Switch to select the execution mode. Possible values are {@link CoreOptions#NEW_MODE} + * and {@link CoreOptions#LEGACY_MODE}. */ - public static final ConfigOption MODE = ConfigOptions - .key("mode") - .defaultValue(FLIP6_MODE) - .withDescription("Switch to select the execution mode. Possible values are 'flip6' and 'old'."); + public static final ConfigOption MODE = key("mode") + .defaultValue(NEW_MODE) + .withDescription("Switch to select the execution mode. Possible values are 'new' and 'legacy'."); } diff --git a/flink-dist/src/main/flink-bin/bin/config.sh b/flink-dist/src/main/flink-bin/bin/config.sh index f111ea707a847..46f06ef678e6d 100755 --- a/flink-dist/src/main/flink-bin/bin/config.sh +++ b/flink-dist/src/main/flink-bin/bin/config.sh @@ -267,7 +267,7 @@ fi # Define FLIP if it is not already set if [ -z "${FLINK_MODE}" ]; then - FLINK_MODE=$(readFromConfig ${KEY_FLINK_MODE} "flip6" "${YAML_CONF}") + FLINK_MODE=$(readFromConfig ${KEY_FLINK_MODE} "new" "${YAML_CONF}") fi diff --git a/flink-dist/src/main/flink-bin/bin/jobmanager.sh b/flink-dist/src/main/flink-bin/bin/jobmanager.sh index 10ff7758e4e4f..f03c0a93f0b0a 100755 --- a/flink-dist/src/main/flink-bin/bin/jobmanager.sh +++ b/flink-dist/src/main/flink-bin/bin/jobmanager.sh @@ -36,7 +36,7 @@ bin=`cd "$bin"; pwd` JOBMANAGER_TYPE=jobmanager -if [[ "${FLINK_MODE}" == "flip6" ]]; then +if [[ "${FLINK_MODE}" == "new" ]]; then JOBMANAGER_TYPE=standalonesession fi diff --git a/flink-dist/src/main/flink-bin/bin/start-cluster.sh b/flink-dist/src/main/flink-bin/bin/start-cluster.sh index 4b6eba128dfb6..068577bc26609 100755 --- a/flink-dist/src/main/flink-bin/bin/start-cluster.sh +++ b/flink-dist/src/main/flink-bin/bin/start-cluster.sh @@ -50,4 +50,4 @@ fi shopt -u nocasematch # Start TaskManager instance(s) -TMSlaves start $FLIP6 +TMSlaves start diff --git a/flink-dist/src/main/flink-bin/bin/stop-cluster.sh b/flink-dist/src/main/flink-bin/bin/stop-cluster.sh index bcd97eb9ed3b8..7eb58be9d999b 100755 --- a/flink-dist/src/main/flink-bin/bin/stop-cluster.sh +++ b/flink-dist/src/main/flink-bin/bin/stop-cluster.sh @@ -22,10 +22,8 @@ bin=`cd "$bin"; pwd` . "$bin"/config.sh -FLIP6=$1 - # Stop TaskManager instance(s) -TMSlaves stop $FLIP6 +TMSlaves stop # Stop JobManager instance(s) shopt -s nocasematch @@ -35,15 +33,15 @@ if [[ $HIGH_AVAILABILITY == "zookeeper" ]]; then if [ ${MASTERS_ALL_LOCALHOST} = true ] ; then for master in ${MASTERS[@]}; do - "$FLINK_BIN_DIR"/jobmanager.sh stop "${FLIP6}" + "$FLINK_BIN_DIR"/jobmanager.sh stop done else for master in ${MASTERS[@]}; do - ssh -n $FLINK_SSH_OPTS $master -- "nohup /bin/bash -l \"${FLINK_BIN_DIR}/jobmanager.sh\" stop \"${FLIP6}\" &" + ssh -n $FLINK_SSH_OPTS $master -- "nohup /bin/bash -l \"${FLINK_BIN_DIR}/jobmanager.sh\" stop &" done fi else - "$FLINK_BIN_DIR"/jobmanager.sh stop "${FLIP6}" + "$FLINK_BIN_DIR"/jobmanager.sh stop fi shopt -u nocasematch diff --git a/flink-dist/src/main/flink-bin/bin/taskmanager.sh b/flink-dist/src/main/flink-bin/bin/taskmanager.sh index daa209caf453b..6baa79a202f92 100755 --- a/flink-dist/src/main/flink-bin/bin/taskmanager.sh +++ b/flink-dist/src/main/flink-bin/bin/taskmanager.sh @@ -34,7 +34,7 @@ bin=`cd "$bin"; pwd` TYPE=taskmanager -if [[ "${FLINK_MODE}" == "flip6" ]]; then +if [[ "${FLINK_MODE}" == "new" ]]; then TYPE=taskexecutor fi diff --git a/flink-dist/src/main/flink-bin/mesos-bin/mesos-appmaster.sh b/flink-dist/src/main/flink-bin/mesos-bin/mesos-appmaster.sh index 818ddcd2e8c66..e629b7e5c2aba 100755 --- a/flink-dist/src/main/flink-bin/mesos-bin/mesos-appmaster.sh +++ b/flink-dist/src/main/flink-bin/mesos-bin/mesos-appmaster.sh @@ -38,7 +38,7 @@ export FLINK_LIB_DIR ENTRY_POINT=org.apache.flink.mesos.runtime.clusterframework.MesosApplicationMasterRunner -if [[ "${FLINK_MODE}" == "flip6" ]]; then +if [[ "${FLINK_MODE}" == "new" ]]; then ENTRY_POINT=org.apache.flink.mesos.entrypoint.MesosSessionClusterEntrypoint fi diff --git a/flink-dist/src/main/flink-bin/mesos-bin/mesos-taskmanager.sh b/flink-dist/src/main/flink-bin/mesos-bin/mesos-taskmanager.sh index 81368c54756c3..1d8d1b05a0777 100755 --- a/flink-dist/src/main/flink-bin/mesos-bin/mesos-taskmanager.sh +++ b/flink-dist/src/main/flink-bin/mesos-bin/mesos-taskmanager.sh @@ -43,7 +43,7 @@ export FLINK_LIB_DIR ENTRY_POINT=org.apache.flink.mesos.runtime.clusterframework.MesosTaskManager -if [[ "${FLINK_MODE}" == "flip6" ]]; then +if [[ "${FLINK_MODE}" == "new" ]]; then ENTRY_POINT=org.apache.flink.mesos.entrypoint.MesosTaskExecutorRunner fi diff --git a/flink-docs/pom.xml b/flink-docs/pom.xml index 8b41229865f4a..41320663c47de 100644 --- a/flink-docs/pom.xml +++ b/flink-docs/pom.xml @@ -92,7 +92,7 @@ under the License. - + org.apache.flink flink-test-utils-junit ${project.version} diff --git a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java index bbb2024c649e4..6760583dcfdea 100644 --- a/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java +++ b/flink-libraries/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/DependencyTest.java @@ -18,7 +18,7 @@ package org.apache.flink.table.client.gateway.local; -import org.apache.flink.client.cli.Flip6DefaultCLI; +import org.apache.flink.client.cli.DefaultCLI; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.Types; @@ -59,7 +59,7 @@ public void testTableSourceFactoryDiscovery() throws Exception { env, Collections.singletonList(dependency), new Configuration(), - new Flip6DefaultCLI(new Configuration())); + new DefaultCLI(new Configuration())); final SessionContext session = new SessionContext("test-session", new Environment()); diff --git a/flink-libraries/flink-streaming-python/src/main/java/org/apache/flink/streaming/python/api/environment/PythonEnvironmentFactory.java b/flink-libraries/flink-streaming-python/src/main/java/org/apache/flink/streaming/python/api/environment/PythonEnvironmentFactory.java index 0f83169112528..8e437fe9131b1 100644 --- a/flink-libraries/flink-streaming-python/src/main/java/org/apache/flink/streaming/python/api/environment/PythonEnvironmentFactory.java +++ b/flink-libraries/flink-streaming-python/src/main/java/org/apache/flink/streaming/python/api/environment/PythonEnvironmentFactory.java @@ -20,6 +20,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; +import org.apache.flink.streaming.api.environment.LegacyLocalStreamEnvironment; import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -63,7 +64,7 @@ public PythonStreamExecutionEnvironment get_execution_environment() { * @return A local execution environment with the specified parallelism. */ public PythonStreamExecutionEnvironment create_local_execution_environment(Configuration config) { - return new PythonStreamExecutionEnvironment(new LocalStreamEnvironment(config), new Path(localTmpPath), tmpDistributedDir, scriptName); + return new PythonStreamExecutionEnvironment(new LegacyLocalStreamEnvironment(config), new Path(localTmpPath), tmpDistributedDir, scriptName); } /** diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java index 6902cb56dcb9a..41e7f4e112c13 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImpl.java @@ -119,7 +119,7 @@ public KvStateLocationOracle getKvStateLocationOracle(JobID jobId) { final KvStateLocationOracle legacyKvStateLocationOracle = kvStateLocationOracles.get(HighAvailabilityServices.DEFAULT_JOB_ID); // we give preference to the oracle registered under the default job id - // to make it work with the pre Flip-6 code paths + // to make it work with the legacy code paths if (legacyKvStateLocationOracle != null) { return legacyKvStateLocationOracle; } else { diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImplTest.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImplTest.java index cd3325501bc77..7e1cc24edb3b5 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImplTest.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/client/proxy/KvStateClientProxyImplTest.java @@ -85,7 +85,7 @@ public void testKvStateLocationOracle() { * will be used for all requests. */ @Test - public void testPreFlip6CodePathPreference() { + public void testLegacyCodePathPreference() { final TestingKvStateLocationOracle kvStateLocationOracle = new TestingKvStateLocationOracle(); kvStateClientProxy.updateKvStateLocationOracle(HighAvailabilityServices.DEFAULT_JOB_ID, kvStateLocationOracle); final JobID jobId = new JobID(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java index e65e952409cee..e70032a92a1e6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java @@ -58,8 +58,8 @@ public interface HighAvailabilityServices extends AutoCloseable { /** * This JobID should be used to identify the old JobManager when using the - * {@link HighAvailabilityServices}. With Flip-6 every JobManager will have a distinct - * JobID assigned. + * {@link HighAvailabilityServices}. With the new mode every JobMaster will have a + * distinct JobID assigned. */ JobID DEFAULT_JOB_ID = new JobID(0L, 0L); @@ -83,7 +83,7 @@ public interface HighAvailabilityServices extends AutoCloseable { * * @param jobID The identifier of the job. * @return Leader retrieval service to retrieve the job manager for the given job - * @deprecated This method should only be used by non Flip-6 code where the JobManager acts as the master. + * @deprecated This method should only be used by the legacy code where the JobManager acts as the master. */ @Deprecated LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java index d922d7c94288b..4998d262dcb4d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SharedSlot.java @@ -58,7 +58,7 @@ public class SharedSlot extends Slot implements LogicalSlot { private final Set subSlots; // ------------------------------------------------------------------------ - // Old Constructors (prior FLIP-6) + // Old Constructors (legacy code) // ------------------------------------------------------------------------ /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java index e69247ebc0f32..386cdcdf308c2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SimpleSlot.java @@ -58,7 +58,7 @@ public class SimpleSlot extends Slot implements LogicalSlot { private volatile Locality locality = Locality.UNCONSTRAINED; // ------------------------------------------------------------------------ - // Old Constructors (prior FLIP-6) + // Old Constructors (legacy mode) // ------------------------------------------------------------------------ /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java index dbd655431fa3b..9c1b627cdeb69 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Slot.java @@ -61,7 +61,7 @@ public abstract class Slot { /** State where all tasks in this slot have been canceled and the slot been given back to the instance */ private static final int RELEASED = 2; - // temporary placeholder for Slots that are not constructed from an AllocatedSlot (prior to FLIP-6) + // temporary placeholder for Slots that are not constructed from an AllocatedSlot (by legacy code) protected static final AllocationID NO_ALLOCATION_ID = new AllocationID(0L, 0L); protected static final SlotRequestId NO_SLOT_REQUEST_ID = new SlotRequestId(0L, 0L); protected static final SlotSharingGroupId NO_SLOT_SHARING_GROUP_ID = new SlotSharingGroupId(0L, 0L); @@ -92,7 +92,7 @@ public abstract class Slot { /** * Base constructor for slots. * - *

This is the old way of constructing slots, prior to the FLIP-6 resource management refactoring. + *

This is the old way of constructing slots by the legacy code * * @param owner The component from which this slot is allocated. * @param location The location info of the TaskManager where the slot was allocated from diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java index baa452f6d122a..8b098ac6255f5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.instance.Instance; import org.apache.flink.runtime.instance.SharedSlot; import org.apache.flink.runtime.jobmaster.SlotRequestId; +import org.apache.flink.runtime.jobmaster.slotpool.SlotPool; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.util.AbstractID; import org.apache.flink.util.FlinkException; @@ -172,7 +173,7 @@ public void lockLocation() throws IllegalStateException { * Locks the location of this slot. The location can be locked only once * and only after a shared slot has been assigned. * - *

Note: This method exists for compatibility reasons with the Flip-6 SlotPool + *

Note: This method exists for compatibility reasons with the new {@link SlotPool}. * * @param taskManagerLocation to lock this co-location constraint to */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 66770c52bdd77..59e5ff0b596f7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -95,7 +95,7 @@ import static org.apache.flink.util.Preconditions.checkState; /** - * Flip-6 based MiniCluster. + * MiniCluster to execute Flink jobs locally. */ public class MiniCluster implements JobExecutorService, AutoCloseableAsync { @@ -580,7 +580,7 @@ public void runDetached(JobGraph job) throws JobExecutionException, InterruptedE throw new JobExecutionException(job.getJobID(), e); } - // we have to allow queued scheduling in Flip-6 mode because we need to request slots + // we have to allow queued scheduling in the new mode because we need to request slots // from the ResourceManager job.setAllowQueuedScheduling(true); @@ -614,7 +614,7 @@ public JobExecutionResult executeJobBlocking(JobGraph job) throws JobExecutionEx throw new JobExecutionException(job.getJobID(), e); } - // we have to allow queued scheduling in Flip-6 mode because we need to request slots + // we have to allow queued scheduling in the new mode because we need to request slots // from the ResourceManager job.setAllowQueuedScheduling(true); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java index 63d3c52242670..430b06bd7d1a5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/query/KvStateRegistry.java @@ -167,7 +167,7 @@ public TaskKvStateRegistry createTaskRegistry(JobID jobId, JobVertexID jobVertex // ------------------------------------------------------------------------ private KvStateRegistryListener getKvStateRegistryListener(JobID jobId) { - // first check whether we are running the pre-Flip-6 code which registers + // first check whether we are running the legacy code which registers // a single listener under HighAvailabilityServices.DEFAULT_JOB_ID KvStateRegistryListener listener = listeners.get(HighAvailabilityServices.DEFAULT_JOB_ID); diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala index 15581b2b647aa..0aaeae304f941 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala @@ -1204,7 +1204,7 @@ class TaskManager( val jobID = jobInformation.getJobId - // Allocation ids do not work properly without flip-6, so we just fake one, based on the jid. + // Allocation ids do not work properly in legacy mode, so we just fake one, based on the jid. val fakeAllocationID = new AllocationID(jobID.getLowerPart, jobID.getUpperPart) val taskLocalStateStore = taskManagerLocalStateStoresManager.localStateStoreForSubtask( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index e87e1ced25b35..18a8ec1f0d8ed 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -65,7 +65,7 @@ import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.InMemorySubmittedJobGraphStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; @@ -116,7 +116,7 @@ /** * Test for the {@link Dispatcher} component. */ -@Category(Flip6.class) +@Category(New.class) public class DispatcherTest extends TestLogger { private static RpcService rpcService; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FileArchivedExecutionGraphStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FileArchivedExecutionGraphStoreTest.java index e6d700e4f9a02..90c5beb145374 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FileArchivedExecutionGraphStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FileArchivedExecutionGraphStoreTest.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.webmonitor.WebMonitorUtils; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; @@ -60,7 +60,7 @@ /** * Tests for the {@link FileArchivedExecutionGraphStore}. */ -@Category(Flip6.class) +@Category(New.class) public class FileArchivedExecutionGraphStoreTest extends TestLogger { private static final List GLOBALLY_TERMINAL_JOB_STATUS = Arrays.stream(JobStatus.values()) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java index e3633935b7d42..5790b8aa636ac 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java @@ -44,7 +44,7 @@ import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.util.TestingFatalErrorHandler; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.After; @@ -72,7 +72,7 @@ /** * Tests for the {@link MiniDispatcher}. */ -@Category(Flip6.class) +@Category(New.class) public class MiniDispatcherTest extends TestLogger { private static final Time timeout = Time.seconds(10L); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerTest.java index 77d12d54f0c6e..1d4babf53f83e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/heartbeat/HeartbeatManagerTest.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.concurrent.ScheduledExecutor; import org.apache.flink.runtime.concurrent.ScheduledExecutorServiceAdapter; import org.apache.flink.runtime.util.DirectExecutorService; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -57,7 +57,7 @@ /** * Tests for the {@link HeartbeatManager}. */ -@Category(Flip6.class) +@Category(New.class) public class HeartbeatManagerTest extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(HeartbeatManagerTest.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServicesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServicesTest.java index c93dbcca20f10..737b5ee2feb0a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServicesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneHaServicesTest.java @@ -113,7 +113,7 @@ public void testJobManagerLeaderRetrieval() throws Exception { * fixed leader session id. */ @Test - public void testJobManagerLeaderRetrievalFlip6() throws Exception { + public void testJobMasterLeaderRetrieval() throws Exception { JobID jobId1 = new JobID(); JobID jobId2 = new JobID(); final String jobManagerAddress1 = "foobar"; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java index aab8132a07b32..067001dd67a13 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java @@ -753,7 +753,7 @@ public void testLocalizedAssignment3() { // check the scheduler's bookkeeping assertEquals(4, testingSlotProvider.getNumberOfLocalizedAssignments()); - // Flip-6 supports host localized assignments which happen in this case because all TaskManagerLocations point to the loopback address + // Flink supports host localized assignments which happen in this case because all TaskManagerLocations point to the loopback address assertTrue(2 == testingSlotProvider.getNumberOfNonLocalizedAssignments() || 2 == testingSlotProvider.getNumberOfHostLocalizedAssignments()); assertEquals(0, testingSlotProvider.getNumberOfUnconstrainedAssignments()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index 62f5ee2173805..2f6168140df72 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -59,7 +59,7 @@ import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.util.TestingFatalErrorHandler; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.hamcrest.Matchers; @@ -86,7 +86,7 @@ /** * Tests for {@link JobMaster}. */ -@Category(Flip6.class) +@Category(New.class) public class JobMasterTest extends TestLogger { @ClassRule diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobResultTest.java index 1c7f0dd4ffa73..3d793374b27e1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobResultTest.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.jobmaster; import org.apache.flink.api.common.JobID; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.SerializedThrowable; import org.apache.flink.util.TestLogger; @@ -33,7 +33,7 @@ /** * Tests for {@link JobResult}. */ -@Category(Flip6.class) +@Category(New.class) public class JobResultTest extends TestLogger { @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/AllocatedSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/AllocatedSlotsTest.java index f97ff4f160ff3..e18412bfedf51 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/AllocatedSlotsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/AllocatedSlotsTest.java @@ -25,7 +25,7 @@ import org.apache.flink.runtime.jobmaster.SlotRequestId; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -36,7 +36,7 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -@Category(Flip6.class) +@Category(New.class) public class AllocatedSlotsTest extends TestLogger { @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/AvailableSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/AvailableSlotsTest.java index c0074ed55ee39..ed090d8327574 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/AvailableSlotsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/AvailableSlotsTest.java @@ -25,7 +25,7 @@ import org.apache.flink.runtime.jobmanager.slots.SlotAndLocality; import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -38,7 +38,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -@Category(Flip6.class) +@Category(New.class) public class AvailableSlotsTest extends TestLogger { static final ResourceProfile DEFAULT_TESTING_PROFILE = new ResourceProfile(1.0, 512); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/DualKeyMapTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/DualKeyMapTest.java index 4504b5a78f2fd..503173a3c23bc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/DualKeyMapTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/DualKeyMapTest.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.jobmaster.slotpool; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.hamcrest.Matchers; @@ -36,7 +36,7 @@ /** * Tests for the {@link DualKeyMap}. */ -@Category(Flip6.class) +@Category(New.class) public class DualKeyMapTest extends TestLogger { @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRpcTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRpcTest.java index 4c736e8ba0284..52d7237bbfd4e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRpcTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolRpcTest.java @@ -46,7 +46,7 @@ import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.clock.Clock; import org.apache.flink.runtime.util.clock.SystemClock; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; @@ -72,7 +72,7 @@ /** * Tests for the SlotPool using a proper RPC setup. */ -@Category(Flip6.class) +@Category(New.class) public class SlotPoolRpcTest extends TestLogger { private static RpcService rpcService; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSchedulingTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSchedulingTestBase.java index b9036c19ced96..75b0d05318e86 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSchedulingTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolSchedulingTestBase.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.After; @@ -39,7 +39,7 @@ /** * Test base for {@link SlotPool} related scheduling test cases. */ -@Category(Flip6.class) +@Category(New.class) public class SlotPoolSchedulingTestBase extends TestLogger { private static final JobID jobId = new JobID(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java index 502b076e5714a..f025bc8a0a443 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolTest.java @@ -45,7 +45,7 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.clock.ManualClock; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; @@ -80,7 +80,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -@Category(Flip6.class) +@Category(New.class) public class SlotPoolTest extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(SlotPoolTest.class); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java index ec6eae2e6ee2d..4c56c63e4e3fa 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotSharingManagerTest.java @@ -29,7 +29,7 @@ import org.apache.flink.runtime.jobmaster.SlotContext; import org.apache.flink.runtime.jobmaster.SlotRequestId; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.AbstractID; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; @@ -52,7 +52,7 @@ /** * Test cases for the {@link SlotSharingManager}. */ -@Category(Flip6.class) +@Category(New.class) public class SlotSharingManagerTest extends TestLogger { private static final SlotSharingGroupId SLOT_SHARING_GROUP_ID = new SlotSharingGroupId(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java index 646827afbedcd..c9ac4c65257f4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/minicluster/MiniClusterITCase.java @@ -27,7 +27,7 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.ScheduleMode; import org.apache.flink.runtime.testtasks.NoOpInvokable; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.BeforeClass; @@ -39,7 +39,7 @@ /** * Integration test cases for the {@link MiniCluster}. */ -@Category(Flip6.class) +@Category(New.class) public class MiniClusterITCase extends TestLogger { private static Configuration configuration; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateRegistryTest.java index 36a85d1873095..c1c56bf250b87 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateRegistryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateRegistryTest.java @@ -217,7 +217,7 @@ public void testKvStateRegistryListenerNotification() { * will be used for all notifications. */ @Test - public void testPreFlip6CodePathPreference() { + public void testLegacyCodePathPreference() { final KvStateRegistry kvStateRegistry = new KvStateRegistry(); final ArrayDeque stateRegistrationNotifications = new ArrayDeque<>(2); final ArrayDeque stateDeregistrationNotifications = new ArrayDeque<>(2); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdServiceTest.java index 301ab46c577e6..25d7b5b989404 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/JobLeaderIdServiceTest.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.jobmaster.JobMasterId; import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -56,7 +56,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -@Category(Flip6.class) +@Category(New.class) public class JobLeaderIdServiceTest extends TestLogger { /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java index a3d17f2433489..7843d7d8e7604 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java @@ -31,7 +31,7 @@ import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandler; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Assert; import org.junit.Test; @@ -45,7 +45,7 @@ /** * resourceManager HA test, including grant leadership and revoke leadership */ -@Category(Flip6.class) +@Category(New.class) public class ResourceManagerHATest extends TestLogger { @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java index 7e51a2ca29ff5..312adafbcc048 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerJobMasterTest.java @@ -43,7 +43,7 @@ import org.apache.flink.runtime.rpc.exceptions.FencingTokenException; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandler; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; import org.junit.After; @@ -60,7 +60,7 @@ import static org.junit.Assert.fail; import static org.mockito.Mockito.*; -@Category(Flip6.class) +@Category(New.class) public class ResourceManagerJobMasterTest extends TestLogger { private TestingRpcService rpcService; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java index e4b93d362773b..d49ba571efff9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTaskExecutorTest.java @@ -40,7 +40,7 @@ import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandler; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; import org.junit.After; @@ -61,7 +61,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -@Category(Flip6.class) +@Category(New.class) public class ResourceManagerTaskExecutorTest extends TestLogger { private final Time timeout = Time.seconds(10L); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java index f5fa89992e152..7dab6854513a8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java @@ -35,7 +35,7 @@ import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandler; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.After; @@ -48,7 +48,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -@Category(Flip6.class) +@Category(New.class) public class ResourceManagerTest extends TestLogger { private TestingRpcService rpcService; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java index 90ed1648ad181..f504d77dd9783 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotManagerTest.java @@ -39,7 +39,7 @@ import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.exceptions.SlotAllocationException; import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -74,7 +74,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -@Category(Flip6.class) +@Category(New.class) public class SlotManagerTest extends TestLogger { /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java index d47ac33850af1..b3e5e91a55e36 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/SlotProtocolTest.java @@ -33,7 +33,7 @@ import org.apache.flink.runtime.taskexecutor.SlotStatus; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.TestLogger; @@ -54,7 +54,7 @@ import static org.mockito.Mockito.timeout; import static org.mockito.Mockito.verify; -@Category(Flip6.class) +@Category(New.class) public class SlotProtocolTest extends TestLogger { private static final long timeout = 10000L; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java index e510798069e9d..88fdeb85d2418 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointITCase.java @@ -43,7 +43,7 @@ import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; @@ -94,7 +94,7 @@ /** * IT cases for {@link RestClient} and {@link RestServerEndpoint}. */ -@Category(Flip6.class) +@Category(New.class) public class RestServerEndpointITCase extends TestLogger { private static final JobID PATH_JOB_ID = new JobID(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java index de32883b78ab0..bdb3f5dc1944d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerEndpointTest.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.rest; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Assume; @@ -44,7 +44,7 @@ /** * Test cases for the {@link RestServerEndpoint}. */ -@Category(Flip6.class) +@Category(New.class) public class RestServerEndpointTest extends TestLogger { @Rule diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/BlobServerPortHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/BlobServerPortHandlerTest.java index 0c2c2eda3d7f6..6c81313c364e8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/BlobServerPortHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/BlobServerPortHandlerTest.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; @@ -48,7 +48,7 @@ /** * Tests for the {@link BlobServerPortHandler}. */ -@Category(Flip6.class) +@Category(New.class) public class BlobServerPortHandlerTest extends TestLogger { private static final int PORT = 64; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandlerTest.java index d69c5d5de1acb..e32e16a6480dc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExecutionResultHandlerTest.java @@ -33,7 +33,7 @@ import org.apache.flink.runtime.rest.messages.job.JobExecutionResultResponseBody; import org.apache.flink.runtime.rest.messages.queue.QueueStatus; import org.apache.flink.runtime.webmonitor.TestingRestfulGateway; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -57,7 +57,7 @@ /** * Tests for {@link JobExecutionResultHandler}. */ -@Category(Flip6.class) +@Category(New.class) public class JobExecutionResultHandlerTest extends TestLogger { private static final JobID TEST_JOB_ID = new JobID(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobSubmitHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobSubmitHandlerTest.java index 2428d38e22cce..ac09186230dc2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobSubmitHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobSubmitHandlerTest.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.rest.messages.job.JobSubmitRequestBody; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; @@ -47,7 +47,7 @@ /** * Tests for the {@link JobSubmitHandler}. */ -@Category(Flip6.class) +@Category(New.class) public class JobSubmitHandlerTest extends TestLogger { @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AbstractMetricsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AbstractMetricsHandlerTest.java index 80a87597be08d..0d018bc9b76f7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AbstractMetricsHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AbstractMetricsHandlerTest.java @@ -35,7 +35,7 @@ import org.apache.flink.runtime.rest.messages.job.metrics.MetricCollectionResponseBody; import org.apache.flink.runtime.rest.messages.job.metrics.MetricsFilterParameter; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Before; @@ -61,7 +61,7 @@ /** * Tests for {@link AbstractMetricsHandler}. */ -@Category(Flip6.class) +@Category(New.class) public class AbstractMetricsHandlerTest extends TestLogger { private static final String TEST_METRIC_NAME = "test_counter"; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/MetricsHandlerTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/MetricsHandlerTestBase.java index 92ee1856093f6..bf37cc95e29c0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/MetricsHandlerTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/MetricsHandlerTestBase.java @@ -29,7 +29,7 @@ import org.apache.flink.runtime.rest.messages.job.metrics.Metric; import org.apache.flink.runtime.rest.messages.job.metrics.MetricCollectionResponseBody; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Before; @@ -51,7 +51,7 @@ /** * Unit test base class for subclasses of {@link AbstractMetricsHandler}. */ -@Category(Flip6.class) +@Category(New.class) public abstract class MetricsHandlerTestBase extends TestLogger { private static final String TEST_METRIC_NAME = "test_counter"; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java index 8d73231350a09..5bc9b60a9b4e3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/MessageParametersTest.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.rest.messages; import org.apache.flink.api.common.JobID; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Assert; @@ -32,7 +32,7 @@ /** * Tests for {@link MessageParameters}. */ -@Category(Flip6.class) +@Category(New.class) public class MessageParametersTest extends TestLogger { @Test public void testResolveUrl() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/RestRequestMarshallingTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/RestRequestMarshallingTestBase.java index 4dc81e8a268e4..c5ae563264c1d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/RestRequestMarshallingTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/RestRequestMarshallingTestBase.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.rest.messages; import org.apache.flink.runtime.rest.util.RestMapperUtils; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; @@ -31,7 +31,7 @@ /** * Test base for verifying that marshalling / unmarshalling REST {@link RequestBody}s work properly. */ -@Category(Flip6.class) +@Category(New.class) public abstract class RestRequestMarshallingTestBase extends TestLogger { /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/RestResponseMarshallingTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/RestResponseMarshallingTestBase.java index 7442dcb4b14bc..fa7ec4c2d62bc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/RestResponseMarshallingTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/RestResponseMarshallingTestBase.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.rest.messages; import org.apache.flink.runtime.rest.util.RestMapperUtils; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JavaType; @@ -35,7 +35,7 @@ /** * Test base for verifying that marshalling / unmarshalling REST {@link ResponseBody}s work properly. */ -@Category(Flip6.class) +@Category(New.class) public abstract class RestResponseMarshallingTestBase extends TestLogger { /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/AbstractMetricsHeadersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/AbstractMetricsHeadersTest.java index 5228e0ee9aa30..0ea2d372c1dba 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/AbstractMetricsHeadersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/AbstractMetricsHeadersTest.java @@ -21,7 +21,7 @@ import org.apache.flink.runtime.rest.HttpMethodWrapper; import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; @@ -36,7 +36,7 @@ /** * Tests for {@link AbstractMetricsHeaders}. */ -@Category(Flip6.class) +@Category(New.class) public class AbstractMetricsHeadersTest extends TestLogger { private AbstractMetricsHeaders metricsHandlerHeaders; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/JobManagerMetricsHeadersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/JobManagerMetricsHeadersTest.java index e87d3deae390c..74c7603062a30 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/JobManagerMetricsHeadersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/JobManagerMetricsHeadersTest.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.rest.messages.job.metrics; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -31,7 +31,7 @@ /** * Tests for {@link JobManagerMetricsHeaders}. */ -@Category(Flip6.class) +@Category(New.class) public class JobManagerMetricsHeadersTest extends TestLogger { private final JobManagerMetricsHeaders jobManagerMetricsHeaders = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/JobMetricsHeadersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/JobMetricsHeadersTest.java index 515c7c4621849..581144252a1f9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/JobMetricsHeadersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/JobMetricsHeadersTest.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.rest.messages.job.metrics; import org.apache.flink.runtime.rest.messages.JobIDPathParameter; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -32,7 +32,7 @@ /** * Tests for {@link JobMetricsHeaders}. */ -@Category(Flip6.class) +@Category(New.class) public class JobMetricsHeadersTest extends TestLogger { private final JobMetricsHeaders jobMetricsHeaders = JobMetricsHeaders.getInstance(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/JobVertexMetricsHeadersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/JobVertexMetricsHeadersTest.java index f20abdb3ea1f8..5dd256704e571 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/JobVertexMetricsHeadersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/JobVertexMetricsHeadersTest.java @@ -20,7 +20,7 @@ import org.apache.flink.runtime.rest.messages.JobIDPathParameter; import org.apache.flink.runtime.rest.messages.JobVertexIdPathParameter; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -33,7 +33,7 @@ /** * Tests for {@link JobVertexMetricsHeaders}. */ -@Category(Flip6.class) +@Category(New.class) public class JobVertexMetricsHeadersTest extends TestLogger { private final JobVertexMetricsHeaders jobVertexMetricsHeaders = JobVertexMetricsHeaders diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameterTest.java index b13cb01f64700..f30132dc391c3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/MetricsFilterParameterTest.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.rest.messages.job.metrics; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Before; @@ -32,7 +32,7 @@ /** * Tests for {@link MetricsFilterParameter}. */ -@Category(Flip6.class) +@Category(New.class) public class MetricsFilterParameterTest extends TestLogger { private MetricsFilterParameter metricsFilterParameter; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/SubtaskMetricsHeadersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/SubtaskMetricsHeadersTest.java index 0f82465e902b0..f99c423515ebd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/SubtaskMetricsHeadersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/SubtaskMetricsHeadersTest.java @@ -21,7 +21,7 @@ import org.apache.flink.runtime.rest.messages.JobIDPathParameter; import org.apache.flink.runtime.rest.messages.JobVertexIdPathParameter; import org.apache.flink.runtime.rest.messages.SubtaskIndexPathParameter; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -34,7 +34,7 @@ /** * Tests for {@link SubtaskMetricsHeaders}. */ -@Category(Flip6.class) +@Category(New.class) public class SubtaskMetricsHeadersTest extends TestLogger { private final SubtaskMetricsHeaders subtaskMetricsHeaders = SubtaskMetricsHeaders.getInstance(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/TaskManagerMetricsHeadersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/TaskManagerMetricsHeadersTest.java index 477e9f8b9accd..66b0d8ecc411e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/TaskManagerMetricsHeadersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/job/metrics/TaskManagerMetricsHeadersTest.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.rest.messages.job.metrics; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerIdPathParameter; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -32,7 +32,7 @@ /** * Tests for {@link TaskManagerMetricsHeaders}. */ -@Category(Flip6.class) +@Category(New.class) public class TaskManagerMetricsHeadersTest extends TestLogger { private final TaskManagerMetricsHeaders taskManagerMetricsHeaders = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerIdPathParameterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerIdPathParameterTest.java index 379fe4c7dc085..9c6291584a0e4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerIdPathParameterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/taskmanager/TaskManagerIdPathParameterTest.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.rest.messages.taskmanager; import org.apache.flink.runtime.clusterframework.types.ResourceID; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Before; @@ -33,7 +33,7 @@ /** * Tests for {@link TaskManagerIdPathParameter}. */ -@Category(Flip6.class) +@Category(New.class) public class TaskManagerIdPathParameterTest extends TestLogger { private TaskManagerIdPathParameter taskManagerIdPathParameter; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java index 1f9d9e3f73736..5f5ba448e3f2a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java @@ -25,7 +25,7 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.runtime.rpc.exceptions.FencingTokenException; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -52,7 +52,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -@Category(Flip6.class) +@Category(New.class) public class AsyncCallsTest extends TestLogger { // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java index f488308b0b7e6..00ff3d7bc5c24 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/FencedRpcEndpointTest.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.exceptions.FencingTokenException; import org.apache.flink.runtime.rpc.exceptions.RpcException; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; @@ -46,7 +46,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -@Category(Flip6.class) +@Category(New.class) public class FencedRpcEndpointTest extends TestLogger { private static final Time timeout = Time.seconds(10L); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java index 017c1f58029c4..78137e86ff76b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcConnectionTest.java @@ -27,7 +27,7 @@ import org.apache.flink.runtime.rpc.akka.AkkaRpcService; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import akka.actor.Terminated; @@ -49,7 +49,7 @@ * This test validates that the RPC service gives a good message when it cannot * connect to an RpcEndpoint. */ -@Category(Flip6.class) +@Category(New.class) public class RpcConnectionTest extends TestLogger { @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java index b5add60b246aa..1ca39496f9227 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcEndpointTest.java @@ -22,7 +22,7 @@ import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.rpc.akka.AkkaRpcService; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import akka.actor.ActorSystem; @@ -42,7 +42,7 @@ /** * Tests for the RpcEndpoint and its self gateways. */ -@Category(Flip6.class) +@Category(New.class) public class RpcEndpointTest extends TestLogger { private static final Time TIMEOUT = Time.seconds(10L); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java index 2530bcea1ee6e..a92235c04f6d0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.rpc.akka.exceptions.AkkaRpcException; import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLogger; @@ -51,7 +51,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -@Category(Flip6.class) +@Category(New.class) public class AkkaRpcActorTest extends TestLogger { // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java index d92e496f20573..d73ee40ec047e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.akka.AkkaUtils; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.concurrent.ScheduledExecutor; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import akka.actor.ActorSystem; @@ -46,7 +46,7 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -@Category(Flip6.class) +@Category(New.class) public class AkkaRpcServiceTest extends TestLogger { // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java index 6dacdfd47dffc..34f8eb85c2103 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java @@ -25,7 +25,7 @@ import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -34,7 +34,7 @@ import static org.junit.Assert.assertTrue; -@Category(Flip6.class) +@Category(New.class) public class MainThreadValidationTest extends TestLogger { @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java index 60068505fa231..02c0094048582 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcGateway; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import akka.actor.ActorSystem; @@ -51,7 +51,7 @@ /** * Tests that akka rpc invocation messages are properly serialized and errors reported */ -@Category(Flip6.class) +@Category(New.class) public class MessageSerializationTest extends TestLogger { private static ActorSystem actorSystem1; private static ActorSystem actorSystem2; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java index 8dcd6420b6b9a..2116c2f36c8a0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java @@ -22,7 +22,7 @@ import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.state.LocalRecoveryConfig; import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; -import org.apache.flink.testutils.category.OldAndFlip6; +import org.apache.flink.testutils.category.LegacyAndNew; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -35,7 +35,7 @@ /** * Tests the network buffer calculation from heap size. */ -@Category(OldAndFlip6.class) +@Category(LegacyAndNew.class) public class NetworkBufferCalculationTest extends TestLogger { /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java index fc8337f086fcb..885d99f616cd9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java @@ -58,7 +58,7 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandler; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.hamcrest.Matchers; @@ -86,7 +86,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -@Category(Flip6.class) +@Category(New.class) public class TaskExecutorITCase extends TestLogger { private final Time timeout = Time.seconds(10L); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 7aae28704a031..465619e278e64 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -88,7 +88,7 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testingUtils.TestingUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandler; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.FlinkException; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; @@ -139,7 +139,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -@Category(Flip6.class) +@Category(New.class) public class TaskExecutorTest extends TestLogger { @Rule diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfigurationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfigurationTest.java index 2699a05da98ac..9cf76f20d5553 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfigurationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfigurationTest.java @@ -20,7 +20,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.testutils.category.OldAndFlip6; +import org.apache.flink.testutils.category.LegacyAndNew; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -31,7 +31,7 @@ /** * Unit test for {@link TaskManagerServicesConfiguration}. */ -@Category(OldAndFlip6.class) +@Category(LegacyAndNew.class) public class TaskManagerServicesConfigurationTest extends TestLogger { /** * Verifies that {@link TaskManagerServicesConfiguration#hasNewNetworkBufConf(Configuration)} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java index f6e7b07e07b9f..097fc61369fb5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesTest.java @@ -20,7 +20,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.testutils.category.OldAndFlip6; +import org.apache.flink.testutils.category.LegacyAndNew; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -35,7 +35,7 @@ /** * Unit test for {@link TaskManagerServices}. */ -@Category(OldAndFlip6.class) +@Category(LegacyAndNew.class) public class TaskManagerServicesTest extends TestLogger { /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java index a22cc6fe5bfe8..43b0be249bced 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TestingTaskExecutorGateway.java @@ -34,7 +34,7 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.StackTraceSampleResponse; import org.apache.flink.runtime.resourcemanager.ResourceManagerId; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.Preconditions; import org.junit.experimental.categories.Category; @@ -45,7 +45,7 @@ /** * Simple {@link TaskExecutorGateway} implementation for testing purposes. */ -@Category(Flip6.class) +@Category(New.class) public class TestingTaskExecutorGateway implements TaskExecutorGateway { private final String address; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TimerServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TimerServiceTest.java index ddaa0a6c537c3..642c7900cd4ea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TimerServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TimerServiceTest.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.taskexecutor.slot; import org.apache.flink.runtime.clusterframework.types.AllocationID; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -39,7 +39,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -@Category(Flip6.class) +@Category(New.class) public class TimerServiceTest extends TestLogger { /** * Test all timeouts registered can be unregistered diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java index 4b73b0925ff50..62ae19b997158 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java @@ -37,7 +37,7 @@ import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; import org.apache.flink.runtime.testingUtils.TestingUtils; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.types.LongValue; import org.apache.flink.util.TestLogger; @@ -53,7 +53,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -@Category(Flip6.class) +@Category(New.class) public class TaskCancelAsyncProducerConsumerITCase extends TestLogger { // The Exceptions thrown by the producer/consumer Threads diff --git a/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala b/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala index 95a2999454a9e..cb6231ad2235f 100644 --- a/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala +++ b/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala @@ -322,7 +322,7 @@ object ScalaShellITCase { @BeforeClass def beforeAll(): Unit = { configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, parallelism) - configuration.setString(CoreOptions.MODE, CoreOptions.OLD_MODE) + configuration.setString(CoreOptions.MODE, CoreOptions.LEGACY_MODE) cluster = Option(new StandaloneMiniCluster(configuration)) } diff --git a/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellLocalStartupITCase.scala b/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellLocalStartupITCase.scala index 34832e729a605..9365948932e2d 100644 --- a/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellLocalStartupITCase.scala +++ b/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellLocalStartupITCase.scala @@ -85,7 +85,7 @@ class ScalaShellLocalStartupITCase extends TestLogger { System.setOut(new PrintStream(baos)) val configuration = new Configuration() - configuration.setString(CoreOptions.MODE, CoreOptions.OLD_MODE) + configuration.setString(CoreOptions.MODE, CoreOptions.LEGACY_MODE) val dir = temporaryFolder.newFolder() BootstrapTools.writeConfiguration(configuration, new File(dir, "flink-conf.yaml")) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LegacyLocalStreamEnvironment.java similarity index 64% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LegacyLocalStreamEnvironment.java index 4cc23fc2ec703..8341ec4c5ca96 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/Flip6LocalStreamEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LegacyLocalStreamEnvironment.java @@ -17,47 +17,46 @@ package org.apache.flink.streaming.api.environment; -import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.Public; import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.minicluster.MiniCluster; -import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; +import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; import org.apache.flink.streaming.api.graph.StreamGraph; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * The Flip6LocalStreamEnvironment is a StreamExecutionEnvironment that runs the program locally, + * The LocalStreamEnvironment is a StreamExecutionEnvironment that runs the program locally, * multi-threaded, in the JVM where the environment is instantiated. It spawns an embedded * Flink cluster in the background and executes the program on that cluster. * *

When this environment is instantiated, it uses a default parallelism of {@code 1}. The default * parallelism can be set via {@link #setParallelism(int)}. + * + *

Local environments can also be instantiated through {@link StreamExecutionEnvironment#createLocalEnvironment()} + * and {@link StreamExecutionEnvironment#createLocalEnvironment(int)}. The former version will pick a + * default parallelism equal to the number of hardware contexts in the local machine. */ -@Internal -public class Flip6LocalStreamEnvironment extends LocalStreamEnvironment { +@Public +public class LegacyLocalStreamEnvironment extends LocalStreamEnvironment { - private static final Logger LOG = LoggerFactory.getLogger(Flip6LocalStreamEnvironment.class); + private static final Logger LOG = LoggerFactory.getLogger(LocalStreamEnvironment.class); - /** - * Creates a new mini cluster stream environment that uses the default configuration. - */ - public Flip6LocalStreamEnvironment() { - this(null); + public LegacyLocalStreamEnvironment() { + this(new Configuration()); } /** - * Creates a new mini cluster stream environment that configures its local executor with the given configuration. + * Creates a new local stream environment that configures its local executor with the given configuration. * * @param config The configuration used to configure the local executor. */ - public Flip6LocalStreamEnvironment(Configuration config) { + public LegacyLocalStreamEnvironment(Configuration config) { super(config); - setParallelism(1); } /** @@ -75,37 +74,28 @@ public JobExecutionResult execute(String jobName) throws Exception { streamGraph.setJobName(jobName); JobGraph jobGraph = streamGraph.getJobGraph(); - jobGraph.setAllowQueuedScheduling(true); Configuration configuration = new Configuration(); configuration.addAll(jobGraph.getJobConfiguration()); + configuration.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, -1L); + configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, jobGraph.getMaximumParallelism()); // add (and override) the settings with what the user defined - configuration.addAll(this.conf); - - configuration.setInteger(RestOptions.REST_PORT, 0); - - MiniClusterConfiguration cfg = new MiniClusterConfiguration.Builder() - .setConfiguration(configuration) - .setNumSlotsPerTaskManager(jobGraph.getMaximumParallelism()) - .build(); + configuration.addAll(getConfiguration()); if (LOG.isInfoEnabled()) { LOG.info("Running job on local embedded Flink mini cluster"); } - MiniCluster miniCluster = new MiniCluster(cfg); - + LocalFlinkMiniCluster exec = new LocalFlinkMiniCluster(configuration, true); try { - miniCluster.start(); - configuration.setInteger(RestOptions.REST_PORT, miniCluster.getRestAddress().getPort()); - - return miniCluster.executeJobBlocking(jobGraph); + exec.start(); + return exec.submitJobAndWait(jobGraph, getConfig().isSysoutLoggingEnabled()); } finally { transformations.clear(); - miniCluster.close(); + exec.stop(); } } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java index a53e2a35cb90e..935c78eff91f4 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironment.java @@ -21,16 +21,19 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.MiniClusterConfiguration; import org.apache.flink.streaming.api.graph.StreamGraph; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; + /** * The LocalStreamEnvironment is a StreamExecutionEnvironment that runs the program locally, * multi-threaded, in the JVM where the environment is instantiated. It spawns an embedded @@ -38,39 +41,38 @@ * *

When this environment is instantiated, it uses a default parallelism of {@code 1}. The default * parallelism can be set via {@link #setParallelism(int)}. - * - *

Local environments can also be instantiated through {@link StreamExecutionEnvironment#createLocalEnvironment()} - * and {@link StreamExecutionEnvironment#createLocalEnvironment(int)}. The former version will pick a - * default parallelism equal to the number of hardware contexts in the local machine. */ @Public public class LocalStreamEnvironment extends StreamExecutionEnvironment { private static final Logger LOG = LoggerFactory.getLogger(LocalStreamEnvironment.class); - /** The configuration to use for the local cluster. */ - protected final Configuration conf; + private final Configuration configuration; /** - * Creates a new local stream environment that uses the default configuration. + * Creates a new mini cluster stream environment that uses the default configuration. */ public LocalStreamEnvironment() { - this(null); + this(new Configuration()); } /** - * Creates a new local stream environment that configures its local executor with the given configuration. + * Creates a new mini cluster stream environment that configures its local executor with the given configuration. * - * @param config The configuration used to configure the local executor. + * @param configuration The configuration used to configure the local executor. */ - public LocalStreamEnvironment(Configuration config) { + public LocalStreamEnvironment(@Nonnull Configuration configuration) { if (!ExecutionEnvironment.areExplicitEnvironmentsAllowed()) { throw new InvalidProgramException( - "The LocalStreamEnvironment cannot be used when submitting a program through a client, " + - "or running in a TestEnvironment context."); + "The LocalStreamEnvironment cannot be used when submitting a program through a client, " + + "or running in a TestEnvironment context."); } + this.configuration = configuration; + setParallelism(1); + } - this.conf = config == null ? new Configuration() : config; + protected Configuration getConfiguration() { + return configuration; } /** @@ -88,28 +90,37 @@ public JobExecutionResult execute(String jobName) throws Exception { streamGraph.setJobName(jobName); JobGraph jobGraph = streamGraph.getJobGraph(); + jobGraph.setAllowQueuedScheduling(true); Configuration configuration = new Configuration(); configuration.addAll(jobGraph.getJobConfiguration()); - configuration.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE, -1L); - configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, jobGraph.getMaximumParallelism()); // add (and override) the settings with what the user defined - configuration.addAll(this.conf); + configuration.addAll(this.configuration); + + configuration.setInteger(RestOptions.REST_PORT, 0); + + MiniClusterConfiguration cfg = new MiniClusterConfiguration.Builder() + .setConfiguration(configuration) + .setNumSlotsPerTaskManager(jobGraph.getMaximumParallelism()) + .build(); if (LOG.isInfoEnabled()) { LOG.info("Running job on local embedded Flink mini cluster"); } - LocalFlinkMiniCluster exec = new LocalFlinkMiniCluster(configuration, true); + MiniCluster miniCluster = new MiniCluster(cfg); + try { - exec.start(); - return exec.submitJobAndWait(jobGraph, getConfig().isSysoutLoggingEnabled()); + miniCluster.start(); + configuration.setInteger(RestOptions.REST_PORT, miniCluster.getRestAddress().getPort()); + + return miniCluster.executeJobBlocking(jobGraph); } finally { transformations.clear(); - exec.stop(); + miniCluster.close(); } } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java index 036cf4dae3195..075a3cd0caa2f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/RemoteStreamEnvironment.java @@ -203,7 +203,7 @@ protected JobExecutionResult executeRemotely(StreamGraph streamGraph, List final ClusterClient client; try { - if (CoreOptions.OLD_MODE.equals(configuration.getString(CoreOptions.MODE))) { + if (CoreOptions.LEGACY_MODE.equals(configuration.getString(CoreOptions.MODE))) { client = new StandaloneClusterClient(configuration); } else { client = new RestClusterClient<>(configuration, "RemoteStreamEnvironment"); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java index c39201c427846..fa81c2773db94 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java @@ -1652,10 +1652,10 @@ public static LocalStreamEnvironment createLocalEnvironment(int parallelism) { public static LocalStreamEnvironment createLocalEnvironment(int parallelism, Configuration configuration) { final LocalStreamEnvironment currentEnvironment; - if (CoreOptions.FLIP6_MODE.equals(configuration.getString(CoreOptions.MODE))) { - currentEnvironment = new Flip6LocalStreamEnvironment(configuration); - } else { + if (CoreOptions.NEW_MODE.equals(configuration.getString(CoreOptions.MODE))) { currentEnvironment = new LocalStreamEnvironment(configuration); + } else { + currentEnvironment = new LegacyLocalStreamEnvironment(configuration); } currentEnvironment.setParallelism(parallelism); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironmentITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironmentITCase.java index f302eda6856fb..84a1ce94f062f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironmentITCase.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/LocalStreamEnvironmentITCase.java @@ -26,7 +26,7 @@ import static org.junit.Assert.assertEquals; /** - * Tests for {@link Flip6LocalStreamEnvironment}. + * Tests for {@link LocalStreamEnvironment}. */ @SuppressWarnings("serial") public class LocalStreamEnvironmentITCase extends TestLogger { @@ -37,7 +37,7 @@ public class LocalStreamEnvironmentITCase extends TestLogger { */ @Test public void testRunIsolatedJob() throws Exception { - Flip6LocalStreamEnvironment env = new Flip6LocalStreamEnvironment(); + LocalStreamEnvironment env = new LocalStreamEnvironment(); assertEquals(1, env.getParallelism()); addSmallBoundedJob(env, 3); @@ -50,7 +50,7 @@ public void testRunIsolatedJob() throws Exception { */ @Test public void testMultipleJobsAfterAnother() throws Exception { - Flip6LocalStreamEnvironment env = new Flip6LocalStreamEnvironment(); + LocalStreamEnvironment env = new LocalStreamEnvironment(); addSmallBoundedJob(env, 3); env.execute(); diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/category/Flip6.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/category/LegacyAndNew.java similarity index 87% rename from flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/category/Flip6.java rename to flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/category/LegacyAndNew.java index f23e5b5876066..180f87eba1eb2 100644 --- a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/category/Flip6.java +++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/category/LegacyAndNew.java @@ -19,7 +19,8 @@ package org.apache.flink.testutils.category; /** - * Category marker interface for Flip-6 tests. + * Category marker interface for tests relevant for the legacy and + * new architecture. */ -public interface Flip6 { +public interface LegacyAndNew { } diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/category/OldAndFlip6.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/category/New.java similarity index 90% rename from flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/category/OldAndFlip6.java rename to flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/category/New.java index fd24534aacd3e..2f8fd44188e43 100644 --- a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/category/OldAndFlip6.java +++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/category/New.java @@ -19,7 +19,7 @@ package org.apache.flink.testutils.category; /** - * Category marker interface for old and Flip-6 tests. + * Category marker interface for tests based on the new architecture. */ -public interface OldAndFlip6 { +public interface New { } diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java index d73f6246b3cad..c53ed8b4d5eeb 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java @@ -19,7 +19,7 @@ package org.apache.flink.test.util; import org.apache.flink.configuration.Configuration; -import org.apache.flink.testutils.category.OldAndFlip6; +import org.apache.flink.testutils.category.LegacyAndNew; import org.apache.flink.util.FileUtils; import org.junit.ClassRule; @@ -56,7 +56,7 @@ * * */ -@Category(OldAndFlip6.class) +@Category(LegacyAndNew.class) public abstract class AbstractTestBase extends TestBaseUtils { private static final int DEFAULT_PARALLELISM = 4; diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java index 545be8659fd16..9b0ac776b810a 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MiniClusterResource.java @@ -55,7 +55,7 @@ public class MiniClusterResource extends ExternalResource { private static final String CODEBASE_KEY = "codebase"; - private static final String FLIP6_CODEBASE = "flip6"; + private static final String NEW_CODEBASE = "new"; private final MiniClusterResourceConfiguration miniClusterResourceConfiguration; @@ -80,7 +80,7 @@ public MiniClusterResource( final boolean enableClusterClient) { this( miniClusterResourceConfiguration, - Objects.equals(FLIP6_CODEBASE, System.getProperty(CODEBASE_KEY)) ? MiniClusterType.FLIP6 : MiniClusterType.OLD, + Objects.equals(NEW_CODEBASE, System.getProperty(CODEBASE_KEY)) ? MiniClusterType.NEW : MiniClusterType.LEGACY, enableClusterClient); } @@ -104,7 +104,7 @@ public int getNumberSlots() { public ClusterClient getClusterClient() { if (!enableClusterClient) { // this check is technically only necessary for legacy clusters - // we still fail here for flip6 to keep the behaviors in sync + // we still fail here to keep the behaviors in sync throw new IllegalStateException("To use the client you must enable it with the constructor."); } @@ -164,18 +164,18 @@ public void after() { private void startJobExecutorService(MiniClusterType miniClusterType) throws Exception { switch (miniClusterType) { - case OLD: - startOldMiniCluster(); + case LEGACY: + startLegacyMiniCluster(); break; - case FLIP6: - startFlip6MiniCluster(); + case NEW: + startMiniCluster(); break; default: throw new FlinkRuntimeException("Unknown MiniClusterType " + miniClusterType + '.'); } } - private void startOldMiniCluster() throws Exception { + private void startLegacyMiniCluster() throws Exception { final Configuration configuration = new Configuration(miniClusterResourceConfiguration.getConfiguration()); configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, miniClusterResourceConfiguration.getNumberTaskManagers()); configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, miniClusterResourceConfiguration.getNumberSlotsPerTaskManager()); @@ -190,7 +190,7 @@ private void startOldMiniCluster() throws Exception { } } - private void startFlip6MiniCluster() throws Exception { + private void startMiniCluster() throws Exception { final Configuration configuration = miniClusterResourceConfiguration.getConfiguration(); // we need to set this since a lot of test expect this because TestBaseUtils.startCluster() @@ -284,7 +284,7 @@ public Time getShutdownTimeout() { * Type of the mini cluster to start. */ public enum MiniClusterType { - OLD, - FLIP6 + LEGACY, + NEW } } diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointIT.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointIT.java index f9edfa6369540..db3313cd7c464 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointIT.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointIT.java @@ -36,7 +36,7 @@ import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; import org.apache.flink.test.util.AbstractTestBase; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.ExceptionUtils; import org.junit.Assume; @@ -67,7 +67,7 @@ * * @see org.apache.flink.runtime.jobmaster.JobMaster */ -@Category(Flip6.class) +@Category(New.class) public class JobMasterTriggerSavepointIT extends AbstractTestBase { private static CountDownLatch invokeLatch; diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java index 4a2219ae24b4a..302fe3e522d3d 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java @@ -41,7 +41,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.util.MiniClusterResource; -import org.apache.flink.testutils.category.Flip6; +import org.apache.flink.testutils.category.New; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; @@ -62,7 +62,7 @@ /** * Tests the availability of accumulator results during runtime. */ -@Category(Flip6.class) +@Category(New.class) public class AccumulatorLiveITCase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(AccumulatorLiveITCase.class); diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java index dd1c39806d3e9..074b72131e5c6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java @@ -81,7 +81,7 @@ public void testProgramWithAutoParallelism() throws Exception { assertEquals(PARALLELISM, resultCollection.size()); } catch (Exception ex) { - if (MINI_CLUSTER_RESOURCE.getMiniClusterType().equals(MiniClusterType.OLD)) { + if (MINI_CLUSTER_RESOURCE.getMiniClusterType().equals(MiniClusterType.LEGACY)) { throw ex; } assertTrue( diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/RemoteEnvironmentITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/RemoteEnvironmentITCase.java index aeff5785d7e33..a3a551c705be1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/RemoteEnvironmentITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/RemoteEnvironmentITCase.java @@ -78,7 +78,7 @@ public class RemoteEnvironmentITCase extends TestLogger { public static void setupCluster() throws Exception { configuration = new Configuration(); - if (CoreOptions.FLIP6_MODE.equals(configuration.getString(CoreOptions.MODE))) { + if (CoreOptions.NEW_MODE.equals(configuration.getString(CoreOptions.MODE))) { configuration.setInteger(WebOptions.PORT, 0); final MiniCluster miniCluster = new MiniCluster( new MiniClusterConfiguration.Builder() @@ -115,7 +115,7 @@ public static void tearDownCluster() throws Exception { */ @Test(expected = FlinkException.class) public void testInvalidAkkaConfiguration() throws Throwable { - assumeTrue(CoreOptions.OLD_MODE.equalsIgnoreCase(configuration.getString(CoreOptions.MODE))); + assumeTrue(CoreOptions.LEGACY_MODE.equalsIgnoreCase(configuration.getString(CoreOptions.MODE))); Configuration config = new Configuration(); config.setString(AkkaOptions.STARTUP_TIMEOUT, INVALID_STARTUP_TIMEOUT); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java index 2b97de89eeeba..d217a2ac13c55 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureBatchRecoveryITCase.java @@ -157,7 +157,7 @@ public static Collection executionMode() { */ public void testJobManagerFailure(String zkQuorum, final File coordinateDir) throws Exception { Configuration config = new Configuration(); - config.setString(CoreOptions.MODE, CoreOptions.OLD_MODE); + config.setString(CoreOptions.MODE, CoreOptions.LEGACY_MODE); config.setString(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER"); config.setString(HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, zkQuorum); config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, FileStateBackendBasePath.getAbsolutePath()); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java index f76375df5200b..b85a410e3a6b1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java @@ -149,7 +149,7 @@ public void testCancelingOnProcessFailure() throws Exception { final Throwable[] errorRef = new Throwable[1]; final Configuration configuration = new Configuration(); - configuration.setString(CoreOptions.MODE, CoreOptions.OLD_MODE); + configuration.setString(CoreOptions.MODE, CoreOptions.LEGACY_MODE); // start the test program, which infinitely blocks Runnable programRunner = new Runnable() { diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java index 69fe7d638ad96..7dc6f0cf1d608 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureBatchRecoveryITCase.java @@ -68,7 +68,7 @@ public static Collection executionMode() { public void testTaskManagerFailure(int jobManagerPort, final File coordinateDir) throws Exception { final Configuration configuration = new Configuration(); - configuration.setString(CoreOptions.MODE, CoreOptions.OLD_MODE); + configuration.setString(CoreOptions.MODE, CoreOptions.LEGACY_MODE); ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment("localhost", jobManagerPort, configuration); env.setParallelism(PARALLELISM); env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 10000)); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java index 1ecbff37dfc3d..766a7993c45d0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java @@ -67,7 +67,7 @@ public void testTaskManagerFailure(int jobManagerPort, final File coordinateDir) final File tempCheckpointDir = tempFolder.newFolder(); final Configuration configuration = new Configuration(); - configuration.setString(CoreOptions.MODE, CoreOptions.OLD_MODE); + configuration.setString(CoreOptions.MODE, CoreOptions.LEGACY_MODE); StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment( "localhost", jobManagerPort, diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java index 72f700a82bcc4..b5c2aafb55fc0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/AbstractOperatorRestoreTestBase.java @@ -144,8 +144,8 @@ private String migrateJob(ClassLoader classLoader, ClusterClient clusterClien } catch (Exception e) { String exceptionString = ExceptionUtils.stringifyException(e); if (!(exceptionString.matches("(.*\n)*.*savepoint for the job .* failed(.*\n)*") // legacy - || exceptionString.matches("(.*\n)*.*Not all required tasks are currently running(.*\n)*") // flip6 - || exceptionString.matches("(.*\n)*.*Checkpoint was declined \\(tasks not ready\\)(.*\n)*"))) { // flip6 + || exceptionString.matches("(.*\n)*.*Not all required tasks are currently running(.*\n)*") // new + || exceptionString.matches("(.*\n)*.*Checkpoint was declined \\(tasks not ready\\)(.*\n)*"))) { // new throw e; } } diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java index 4d2aaa02efe34..37b8d410a5d9d 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/TestingYarnClusterDescriptor.java @@ -36,7 +36,7 @@ * flink-yarn-tests-X-tests.jar and the flink-runtime-X-tests.jar to the set of files which * are shipped to the yarn cluster. This is necessary to load the testing classes. */ -public class TestingYarnClusterDescriptor extends YarnClusterDescriptor { +public class TestingYarnClusterDescriptor extends LegacyYarnClusterDescriptor { public TestingYarnClusterDescriptor( Configuration configuration, diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java index f9c03f937849f..18bcfebed329c 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java @@ -106,7 +106,7 @@ public static void teardown() throws Exception { */ @Test public void testMultipleAMKill() throws Exception { - assumeTrue("This test only works with the old actor based code.", !flip6); + assumeTrue("This test only works with the old actor based code.", !isNewMode); final int numberKillingAttempts = numberApplicationAttempts - 1; String confDirPath = System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR); final Configuration configuration = GlobalConfiguration.loadConfiguration(); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java index ef6706ad5fe36..758a09866d0b7 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNITCase.java @@ -57,15 +57,15 @@ public void testPerJobMode() throws Exception { configuration.setString(AkkaOptions.ASK_TIMEOUT, "30 s"); final YarnClient yarnClient = getYarnClient(); - try (final Flip6YarnClusterDescriptor flip6YarnClusterDescriptor = new Flip6YarnClusterDescriptor( + try (final YarnClusterDescriptor yarnClusterDescriptor = new YarnClusterDescriptor( configuration, getYarnConfiguration(), System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR), yarnClient, true)) { - flip6YarnClusterDescriptor.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); - flip6YarnClusterDescriptor.addShipFiles(Arrays.asList(flinkLibFolder.listFiles())); + yarnClusterDescriptor.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath())); + yarnClusterDescriptor.addShipFiles(Arrays.asList(flinkLibFolder.listFiles())); final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder() .setMasterMemoryMB(768) @@ -87,7 +87,7 @@ public void testPerJobMode() throws Exception { jobGraph.addJar(new org.apache.flink.core.fs.Path(testingJar.toURI())); - ClusterClient clusterClient = flip6YarnClusterDescriptor.deployJobCluster( + ClusterClient clusterClient = yarnClusterDescriptor.deployJobCluster( clusterSpecification, jobGraph, true); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java index d00a9c447d135..37676296e4443 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java @@ -102,7 +102,7 @@ public static void setup() { */ @Test public void testClientStartup() throws IOException { - assumeTrue("Flip-6 does not start TMs upfront.", !flip6); + assumeTrue("The new mode does not start TMs upfront.", !isNewMode); LOG.info("Starting testClientStartup()"); runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(), "-n", "1", @@ -192,7 +192,7 @@ public void perJobYarnClusterOffHeap() throws IOException { */ @Test(timeout = 100000) // timeout after 100 seconds public void testTaskManagerFailure() throws Exception { - assumeTrue("Flip-6 does not start TMs upfront.", !flip6); + assumeTrue("The new mode does not start TMs upfront.", !isNewMode); LOG.info("Starting testTaskManagerFailure()"); Runner runner = startWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(), "-t", flinkLibFolder.getAbsolutePath(), "-n", "1", diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java index 464e73c873e4e..d9b02fb359036 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java @@ -99,7 +99,7 @@ public void testDetachedMode() throws InterruptedException, IOException { runner.join(); checkForLogString("The Flink YARN client has been started in detached mode"); - if (!flip6) { + if (!isNewMode) { LOG.info("Waiting until two containers are running"); // wait until two containers are running while (getRunningContainers() < 2) { @@ -241,7 +241,7 @@ public void testJavaAPI() throws Exception { String confDirPath = System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR); Configuration configuration = GlobalConfiguration.loadConfiguration(); - try (final AbstractYarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( + try (final AbstractYarnClusterDescriptor clusterDescriptor = new LegacyYarnClusterDescriptor( configuration, getYarnConfiguration(), confDirPath, diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java index 2a1b099399ac0..c5d683ea03e3c 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java @@ -90,7 +90,7 @@ public void testFlinkContainerMemory() throws Exception { configuration.setLong(TaskManagerOptions.NETWORK_BUFFERS_MEMORY_MAX, (4L << 20)); final YarnConfiguration yarnConfiguration = getYarnConfiguration(); - final Flip6YarnClusterDescriptor clusterDescriptor = new Flip6YarnClusterDescriptor( + final YarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor( configuration, yarnConfiguration, CliFrontend.getConfigurationDirectoryFromEnv(), diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java index 73abc874b915f..421e4c0ebfc9b 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java @@ -81,7 +81,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.regex.Pattern; -import static org.apache.flink.configuration.CoreOptions.OLD_MODE; +import static org.apache.flink.configuration.CoreOptions.LEGACY_MODE; /** * This base class allows to use the MiniYARNCluster. @@ -153,7 +153,7 @@ public abstract class YarnTestBase extends TestLogger { protected org.apache.flink.configuration.Configuration flinkConfiguration; - protected boolean flip6; + protected boolean isNewMode; static { YARN_CONFIGURATION = new YarnConfiguration(); @@ -220,7 +220,7 @@ public void checkClusterEmpty() throws IOException, YarnException { } flinkConfiguration = new org.apache.flink.configuration.Configuration(globalConfiguration); - flip6 = CoreOptions.FLIP6_MODE.equalsIgnoreCase(flinkConfiguration.getString(CoreOptions.MODE)); + isNewMode = CoreOptions.NEW_MODE.equalsIgnoreCase(flinkConfiguration.getString(CoreOptions.MODE)); } @Nullable @@ -528,7 +528,7 @@ private static void start(YarnConfiguration conf, String principal, String keyta globalConfiguration.setString(SecurityOptions.KERBEROS_LOGIN_KEYTAB.key(), keytab); globalConfiguration.setString(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL.key(), principal); - globalConfiguration.setString(CoreOptions.MODE.key(), OLD_MODE); + globalConfiguration.setString(CoreOptions.MODE.key(), LEGACY_MODE); BootstrapTools.writeConfiguration( globalConfiguration, diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java index c80818a2e4b51..ef266f056ba07 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/AbstractYarnClusterDescriptor.java @@ -1042,7 +1042,7 @@ public ApplicationReport startAppMaster( LOG.debug("Application State: {}", appState); switch(appState) { case FAILED: - case FINISHED: //TODO: the finished state may be valid in flip-6 + case FINISHED: case KILLED: throw new YarnDeploymentException("The YARN application unexpectedly switched to state " + appState + " during deployment. \n" + diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/LegacyYarnClusterDescriptor.java similarity index 55% rename from flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java rename to flink-yarn/src/main/java/org/apache/flink/yarn/LegacyYarnClusterDescriptor.java index 1374ca2c4419d..443e6a53f370e 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/Flip6YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/LegacyYarnClusterDescriptor.java @@ -18,14 +18,10 @@ package org.apache.flink.yarn; -import org.apache.flink.client.deployment.ClusterDeploymentException; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint; -import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; @@ -33,12 +29,11 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; /** - * Implementation of {@link org.apache.flink.yarn.AbstractYarnClusterDescriptor} which is used to start the - * new application master for a job under flip-6. + * Legacy implementation of {@link AbstractYarnClusterDescriptor} which starts an {@link YarnApplicationMasterRunner}. */ -public class Flip6YarnClusterDescriptor extends AbstractYarnClusterDescriptor { +public class LegacyYarnClusterDescriptor extends AbstractYarnClusterDescriptor { - public Flip6YarnClusterDescriptor( + public LegacyYarnClusterDescriptor( Configuration flinkConfiguration, YarnConfiguration yarnConfiguration, String configurationDirectory, @@ -54,46 +49,30 @@ public Flip6YarnClusterDescriptor( @Override protected String getYarnSessionClusterEntrypoint() { - return YarnSessionClusterEntrypoint.class.getName(); + return YarnApplicationMasterRunner.class.getName(); } @Override protected String getYarnJobClusterEntrypoint() { - return YarnJobClusterEntrypoint.class.getName(); + throw new UnsupportedOperationException("The old Yarn descriptor does not support proper per-job mode."); } @Override - public ClusterClient deployJobCluster( - ClusterSpecification clusterSpecification, - JobGraph jobGraph, - boolean detached) throws ClusterDeploymentException { - - // this is required to work with Flip-6 because the slots are allocated - // lazily - jobGraph.setAllowQueuedScheduling(true); - - try { - return deployInternal( - clusterSpecification, - "Flink per-job cluster", - getYarnJobClusterEntrypoint(), - jobGraph, - detached); - } catch (Exception e) { - throw new ClusterDeploymentException("Could not deploy Yarn job cluster.", e); - } + public YarnClusterClient deployJobCluster( + ClusterSpecification clusterSpecification, + JobGraph jobGraph, + boolean detached) { + throw new UnsupportedOperationException("Cannot deploy a per-job yarn cluster yet."); } @Override - protected ClusterClient createYarnClusterClient( - AbstractYarnClusterDescriptor descriptor, - int numberTaskManagers, - int slotsPerTaskManager, - ApplicationReport report, - Configuration flinkConfiguration, - boolean perJobCluster) throws Exception { - return new RestClusterClient<>( + protected ClusterClient createYarnClusterClient(AbstractYarnClusterDescriptor descriptor, int numberTaskManagers, int slotsPerTaskManager, ApplicationReport report, Configuration flinkConfiguration, boolean perJobCluster) throws Exception { + return new YarnClusterClient( + descriptor, + numberTaskManagers, + slotsPerTaskManager, + report, flinkConfiguration, - report.getApplicationId()); + perJobCluster); } } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index 8625cee8240c6..3dff72fb8cff8 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -18,10 +18,14 @@ package org.apache.flink.yarn; +import org.apache.flink.client.deployment.ClusterDeploymentException; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint; +import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; @@ -29,7 +33,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration; /** - * Default implementation of {@link AbstractYarnClusterDescriptor} which starts an {@link YarnApplicationMasterRunner}. + * Implementation of {@link AbstractYarnClusterDescriptor} which is used to start the + * application master. */ public class YarnClusterDescriptor extends AbstractYarnClusterDescriptor { @@ -49,30 +54,45 @@ public YarnClusterDescriptor( @Override protected String getYarnSessionClusterEntrypoint() { - return YarnApplicationMasterRunner.class.getName(); + return YarnSessionClusterEntrypoint.class.getName(); } @Override protected String getYarnJobClusterEntrypoint() { - throw new UnsupportedOperationException("The old Yarn descriptor does not support proper per-job mode."); + return YarnJobClusterEntrypoint.class.getName(); } @Override - public YarnClusterClient deployJobCluster( - ClusterSpecification clusterSpecification, - JobGraph jobGraph, - boolean detached) { - throw new UnsupportedOperationException("Cannot deploy a per-job yarn cluster yet."); + public ClusterClient deployJobCluster( + ClusterSpecification clusterSpecification, + JobGraph jobGraph, + boolean detached) throws ClusterDeploymentException { + + // this is required because the slots are allocated lazily + jobGraph.setAllowQueuedScheduling(true); + + try { + return deployInternal( + clusterSpecification, + "Flink per-job cluster", + getYarnJobClusterEntrypoint(), + jobGraph, + detached); + } catch (Exception e) { + throw new ClusterDeploymentException("Could not deploy Yarn job cluster.", e); + } } @Override - protected ClusterClient createYarnClusterClient(AbstractYarnClusterDescriptor descriptor, int numberTaskManagers, int slotsPerTaskManager, ApplicationReport report, Configuration flinkConfiguration, boolean perJobCluster) throws Exception { - return new YarnClusterClient( - descriptor, - numberTaskManagers, - slotsPerTaskManager, - report, + protected ClusterClient createYarnClusterClient( + AbstractYarnClusterDescriptor descriptor, + int numberTaskManagers, + int slotsPerTaskManager, + ApplicationReport report, + Configuration flinkConfiguration, + boolean perJobCluster) throws Exception { + return new RestClusterClient<>( flinkConfiguration, - perJobCluster); + report.getApplicationId()); } } diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java index 446377ffab728..16abffa1245a7 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/cli/FlinkYarnSessionCli.java @@ -40,7 +40,7 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.yarn.AbstractYarnClusterDescriptor; -import org.apache.flink.yarn.Flip6YarnClusterDescriptor; +import org.apache.flink.yarn.LegacyYarnClusterDescriptor; import org.apache.flink.yarn.YarnClusterDescriptor; import org.apache.flink.yarn.configuration.YarnConfigOptions; @@ -158,7 +158,7 @@ public class FlinkYarnSessionCli extends AbstractCustomCommandLine1.6.5 1.3 false - - org.apache.flink.testutils.category.Flip6 - old + + org.apache.flink.testutils.category.New + legacy - flip6 + new