From f1867d316860033af5e83ab13145ed775edb12b0 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Mon, 4 Apr 2022 12:37:26 +0200 Subject: [PATCH] [FLINK-27045][tests] Remove shared executor --- .../ApplicationDispatcherBootstrapITCase.java | 13 +- .../apache/flink/testutils/TestingUtils.java | 18 +- .../util/concurrent/FutureUtilsTest.java | 35 ++-- ...ipleComponentLeaderElectionDriverTest.java | 8 +- .../jobmanager/JMXJobManagerMetricTest.java | 9 +- .../AbstractQueryableStateTestBase.java | 3 +- .../handlers/JarHandlerParameterTest.java | 4 - .../webmonitor/handlers/JarHandlerTest.java | 11 +- .../webmonitor/handlers/JarHandlers.java | 7 +- .../handlers/JarPlanHandlerParameterTest.java | 10 +- .../handlers/JarRunHandlerParameterTest.java | 11 +- .../handlers/JarSubmissionITCase.java | 13 +- .../CheckpointCoordinatorFailureTest.java | 18 +- .../CheckpointCoordinatorMasterHooksTest.java | 22 ++- .../CheckpointCoordinatorRestoringTest.java | 64 +++--- .../checkpoint/CheckpointCoordinatorTest.java | 182 ++++++++---------- .../CheckpointCoordinatorTestingUtils.java | 30 ++- .../CheckpointCoordinatorTriggeringTest.java | 45 +++-- .../CheckpointSettingsSerializableTest.java | 9 +- .../CheckpointStateRestoreTest.java | 19 +- .../CheckpointStatsTrackerTest.java | 14 +- .../DefaultCheckpointPlanCalculatorTest.java | 14 +- .../checkpoint/DefaultCheckpointPlanTest.java | 15 +- ...ultSchedulerCheckpointCoordinatorTest.java | 14 +- ...overStrategyCheckpointCoordinatorTest.java | 11 +- .../StateAssignmentOperationTest.java | 11 +- .../VertexFinishedStateCheckerTest.java | 12 +- .../TaskDeploymentDescriptorFactoryTest.java | 9 +- .../ExecutionGraphInfoStoreTestUtils.java | 23 ++- .../FileExecutionGraphInfoStoreTest.java | 36 +++- .../dispatcher/JobDispatcherITCase.java | 8 +- .../MemoryExecutionGraphInfoStoreTest.java | 13 +- .../runner/DefaultDispatcherRunnerITCase.java | 8 +- .../ZooKeeperDefaultDispatcherRunnerTest.java | 8 +- .../AllVerticesIteratorTest.java | 12 +- .../ArchivedExecutionGraphTest.java | 12 +- ...DefaultExecutionGraphConstructionTest.java | 11 +- .../DefaultExecutionGraphDeploymentTest.java | 31 ++- ...oymentWithSmallBlobCacheSizeLimitTest.java | 4 +- .../DefaultExecutionGraphRescalingTest.java | 25 ++- .../EdgeManagerBuildUtilTest.java | 9 +- .../executiongraph/EdgeManagerTest.java | 12 +- .../ExecutionGraphCoLocationRestartTest.java | 14 +- .../ExecutionGraphFinishTest.java | 14 +- .../ExecutionGraphPartitionReleaseTest.java | 13 +- .../ExecutionGraphRestartTest.java | 38 +++- .../ExecutionGraphSuspendTest.java | 19 +- .../ExecutionGraphTestUtils.java | 26 +-- .../ExecutionGraphVariousFailuesTest.java | 14 +- .../ExecutionJobVertexTest.java | 12 +- .../ExecutionPartitionLifecycleTest.java | 13 +- .../runtime/executiongraph/ExecutionTest.java | 31 ++- .../ExecutionVertexCancelTest.java | 11 +- .../executiongraph/ExecutionVertexTest.java | 20 +- .../executiongraph/FinalizeOnMasterTest.java | 15 +- .../IntermediateResultPartitionTest.java | 24 ++- .../executiongraph/PointwisePatternTest.java | 9 +- .../RemoveCachedShuffleDescriptorTest.java | 10 +- .../TestingDefaultExecutionGraphBuilder.java | 32 +-- .../executiongraph/VertexSlotSharingTest.java | 9 +- .../heartbeat/HeartbeatManagerTest.java | 32 +-- .../AbstractHAJobRunITCase.java | 10 +- .../embedded/EmbeddedLeaderServiceTest.java | 10 +- .../consumer/RemoteInputChannelTest.java | 9 +- .../UpdatePartitionConsumersTest.java | 14 +- .../JobMasterPartitionReleaseTest.java | 2 - .../runtime/jobmaster/JobMasterTest.java | 1 - ...estingJobManagerSharedServicesBuilder.java | 24 +-- .../jobmaster/utils/JobMasterBuilder.java | 7 +- .../LeaderChangeClusterComponentsTest.java | 9 +- .../leaderelection/LeaderElectionTest.java | 9 +- .../ZooKeeperLeaderRetrievalTest.java | 9 +- .../OperatorCoordinatorSchedulerTest.java | 15 +- .../RetryingRegistrationTest.java | 15 +- .../ResourceManagerTaskExecutorTest.java | 9 +- .../resourcemanager/ResourceManagerTest.java | 3 +- .../DeclarativeSlotManagerBuilder.java | 15 +- .../DeclarativeSlotManagerTest.java | 25 ++- .../DefaultSlotStatusSyncerTest.java | 15 +- .../FineGrainedSlotManagerTestBase.java | 11 +- .../TaskExecutorManagerBuilder.java | 12 +- .../slotmanager/TaskExecutorManagerTest.java | 15 +- .../MockResourceManagerRuntimeServices.java | 3 +- .../runtime/rest/RestClientMultipartTest.java | 4 +- .../flink/runtime/rest/RestClientTest.java | 10 +- .../rest/RestExternalHandlersITCase.java | 16 +- .../rest/RestServerEndpointITCase.java | 17 +- .../runtime/rest/RestServerSSLAuthITCase.java | 3 +- .../handler/job/JobConfigHandlerTest.java | 3 +- .../handler/job/JobExceptionsHandlerTest.java | 3 +- .../handler/job/JobSubmitHandlerTest.java | 12 +- ...btaskCurrentAttemptDetailsHandlerTest.java | 6 +- ...ecutionAttemptAccumulatorsHandlerTest.java | 4 +- ...askExecutionAttemptDetailsHandlerTest.java | 6 +- .../AggregatingMetricsHandlerTestBase.java | 9 +- .../DefaultExecutionGraphFactoryTest.java | 10 +- ...DefaultOperatorCoordinatorHandlerTest.java | 9 +- .../DefaultSchedulerBatchSchedulingTest.java | 10 +- .../scheduler/DefaultSchedulerTest.java | 15 +- ...ToInputsLocationsRetrieverAdapterTest.java | 23 ++- ...InputPreferredSlotSharingStrategyTest.java | 11 +- .../scheduler/SchedulerTestingUtils.java | 63 ++++-- .../SsgNetworkMemoryCalculationUtilsTest.java | 14 +- .../adapter/DefaultExecutionTopologyTest.java | 16 +- .../DefaultSchedulingPipelinedRegionTest.java | 11 +- .../adaptive/AdaptiveSchedulerBuilder.java | 23 +-- .../adaptive/AdaptiveSchedulerTest.java | 100 ++++++---- .../scheduler/adaptive/ExecutingTest.java | 16 +- .../AdaptiveBatchSchedulerTest.java | 12 +- .../AdaptiveBatchSchedulerTestUtils.java | 8 +- .../ForwardGroupComputeUtilTest.java | 10 +- .../benchmark/SchedulerBenchmarkBase.java | 7 +- .../benchmark/SchedulerBenchmarkUtils.java | 3 +- .../e2e/SchedulerEndToEndBenchmarkBase.java | 7 +- .../BuildExecutionGraphBenchmark.java | 4 +- .../FailureHandlingResultSnapshotTest.java | 11 +- .../RootExceptionHistoryEntryTest.java | 11 +- ...PipelinedRegionSchedulingStrategyTest.java | 19 +- ...ExecutionDeploymentReconciliationTest.java | 11 +- ...TaskExecutorOperatorEventHandlingTest.java | 10 +- .../TaskExecutorPartitionLifecycleTest.java | 9 +- .../TaskExecutorSlotLifetimeTest.java | 11 +- .../TaskExecutorSubmissionTest.java | 29 ++- .../taskexecutor/TaskExecutorTest.java | 78 +++++--- .../TaskManagerServicesBuilder.java | 12 +- .../TaskSubmissionTestEnvironment.java | 21 +- .../slot/TaskSlotTableImplTest.java | 13 +- .../taskexecutor/slot/TaskSlotUtils.java | 29 +-- ...TaskCancelAsyncProducerConsumerITCase.java | 11 +- .../flink/runtime/taskmanager/TaskTest.java | 102 ++++++---- .../runtime/taskmanager/TestTaskBuilder.java | 9 +- ...bstractUdfStreamOperatorLifecycleTest.java | 19 +- .../partitioner/RescalePartitionerTest.java | 9 +- .../tasks/StreamTaskCancellationTest.java | 19 +- .../tasks/StreamTaskTerminationTest.java | 9 +- .../runtime/tasks/StreamTaskTest.java | 32 ++- .../accumulators/AccumulatorLiveITCase.java | 9 +- .../test/checkpointing/RescalingITCase.java | 9 +- .../test/checkpointing/SavepointITCase.java | 10 +- ...ManagerHAProcessFailureRecoveryITCase.java | 11 +- .../ProcessFailureCancelingITCase.java | 8 +- .../AbstractOperatorRestoreTestBase.java | 17 +- .../flink/yarn/YarnConfigurationITCase.java | 4 +- tools/maven/suppressions-runtime.xml | 2 +- 144 files changed, 1643 insertions(+), 807 deletions(-) diff --git a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java index 47036fcf3528e..3a7cf6b269904 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java +++ b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java @@ -55,16 +55,19 @@ import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorExtension; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; import java.time.Duration; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; import java.util.function.Supplier; import static org.assertj.core.api.Assertions.assertThat; @@ -73,6 +76,10 @@ @ExtendWith(TestLoggerExtension.class) public class ApplicationDispatcherBootstrapITCase { + @RegisterExtension + static final TestExecutorExtension EXECUTOR_EXTENSION = + TestingUtils.defaultExecutorExtension(); + private static Supplier createApplicationModeDispatcherResourceManagerComponentFactorySupplier( Configuration configuration, PackagedProgram program) { @@ -103,7 +110,7 @@ public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exc .setConfiguration(configuration) .build(); final EmbeddedHaServicesWithLeadershipControl haServices = - new EmbeddedHaServicesWithLeadershipControl(TestingUtils.defaultExecutor()); + new EmbeddedHaServicesWithLeadershipControl(EXECUTOR_EXTENSION.getExecutor()); final TestingMiniCluster.Builder clusterBuilder = TestingMiniCluster.newBuilder(clusterConfiguration) .setHighAvailabilityServicesSupplier(() -> haServices) @@ -170,7 +177,7 @@ public void testDirtyJobResultRecoveryInApplicationMode() throws Exception { TestingJobResultStore.createSuccessfulJobResult( ApplicationDispatcherBootstrap.ZERO_JOB_ID))); final EmbeddedHaServicesWithLeadershipControl haServices = - new EmbeddedHaServicesWithLeadershipControl(TestingUtils.defaultExecutor()) { + new EmbeddedHaServicesWithLeadershipControl(EXECUTOR_EXTENSION.getExecutor()) { @Override public JobResultStore getJobResultStore() { @@ -223,7 +230,7 @@ public void testSubmitFailedJobOnApplicationError() throws Exception { .setConfiguration(configuration) .build(); final EmbeddedHaServicesWithLeadershipControl haServices = - new EmbeddedHaServicesWithLeadershipControl(TestingUtils.defaultExecutor()); + new EmbeddedHaServicesWithLeadershipControl(EXECUTOR_EXTENSION.getExecutor()); final TestingMiniCluster.Builder clusterBuilder = TestingMiniCluster.newBuilder(clusterConfiguration) .setHighAvailabilityServicesSupplier(() -> haServices) diff --git a/flink-core/src/test/java/org/apache/flink/testutils/TestingUtils.java b/flink-core/src/test/java/org/apache/flink/testutils/TestingUtils.java index a0c2fc8437f9b..202f4754b1e7c 100644 --- a/flink-core/src/test/java/org/apache/flink/testutils/TestingUtils.java +++ b/flink-core/src/test/java/org/apache/flink/testutils/TestingUtils.java @@ -19,8 +19,8 @@ package org.apache.flink.testutils; import org.apache.flink.api.common.time.Time; -import org.apache.flink.util.concurrent.ScheduledExecutor; -import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; +import org.apache.flink.testutils.executor.TestExecutorExtension; +import org.apache.flink.testutils.executor.TestExecutorResource; import java.time.Duration; import java.util.UUID; @@ -35,8 +35,6 @@ public class TestingUtils { public static final Time TIMEOUT = Time.minutes(1L); public static final Duration DEFAULT_AKKA_ASK_TIMEOUT = Duration.ofSeconds(200); - private static ScheduledExecutorService sharedExecutorInstance; - public static Time infiniteTime() { return Time.milliseconds(Integer.MAX_VALUE); } @@ -47,16 +45,12 @@ public static Duration infiniteDuration() { return Duration.ofDays(365L); } - public static synchronized ScheduledExecutorService defaultExecutor() { - if (sharedExecutorInstance == null || sharedExecutorInstance.isShutdown()) { - sharedExecutorInstance = Executors.newSingleThreadScheduledExecutor(); - } - - return sharedExecutorInstance; + public static TestExecutorExtension defaultExecutorExtension() { + return new TestExecutorExtension<>(Executors::newSingleThreadScheduledExecutor); } - public static ScheduledExecutor defaultScheduledExecutor() { - return new ScheduledExecutorServiceAdapter(defaultExecutor()); + public static TestExecutorResource defaultExecutorResource() { + return new TestExecutorResource<>(Executors::newSingleThreadScheduledExecutor); } public static UUID zeroUUID() { diff --git a/flink-core/src/test/java/org/apache/flink/util/concurrent/FutureUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/concurrent/FutureUtilsTest.java index 901e1adb1eebd..1e730ed7bbb4d 100644 --- a/flink-core/src/test/java/org/apache/flink/util/concurrent/FutureUtilsTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/concurrent/FutureUtilsTest.java @@ -41,7 +41,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; @@ -71,8 +70,8 @@ public class FutureUtilsTest extends TestLogger { @ClassRule - public static final TestExecutorResource TEST_EXECUTOR_RESOURCE = - new TestExecutorResource<>(Executors::newSingleThreadScheduledExecutor); + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); /** Tests that we can retry an operation. */ @Test @@ -91,9 +90,9 @@ public void testRetrySuccess() throws Exception { new FlinkException("Test exception")); } }, - TestingUtils.defaultExecutor()), + EXECUTOR_RESOURCE.getExecutor()), retries, - TestingUtils.defaultExecutor()); + EXECUTOR_RESOURCE.getExecutor()); assertTrue(retryFuture.get()); assertEquals(retries, atomicInteger.get()); @@ -110,7 +109,7 @@ public void testRetryFailureFixedRetries() throws Throwable { FutureUtils.completedExceptionally( new FlinkException("Test exception")), retries, - TestingUtils.defaultExecutor()); + EXECUTOR_RESOURCE.getExecutor()); try { retryFuture.get(); @@ -145,9 +144,9 @@ public void testRetryCancellation() throws Exception { throw new CompletionException( new FlinkException("Test exception")); }, - TestingUtils.defaultExecutor()), + EXECUTOR_RESOURCE.getExecutor()), retries, - TestingUtils.defaultExecutor()); + EXECUTOR_RESOURCE.getExecutor()); // await that we have failed once notificationLatch.await(); @@ -191,12 +190,12 @@ public void testStopAtNonRetryableException() { new FlinkException("Test exception")); } }, - TestingUtils.defaultExecutor()), + EXECUTOR_RESOURCE.getExecutor()), retries, throwable -> ExceptionUtils.findThrowable(throwable, FlinkException.class) .isPresent(), - TestingUtils.defaultExecutor()); + EXECUTOR_RESOURCE.getExecutor()); try { retryFuture.get(); @@ -216,7 +215,7 @@ public void testRetryWithDelayRetryStrategyFailure() throws Throwable { FutureUtils.completedExceptionally( new FlinkException("Test exception")), new FixedRetryStrategy(3, Duration.ofMillis(1L)), - TestingUtils.defaultScheduledExecutor()); + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor())); try { retryFuture.get(TestingUtils.TIMEOUT.toMilliseconds(), TimeUnit.MILLISECONDS); @@ -248,7 +247,7 @@ public void testRetryWithDelayRetryStrategy() throws Exception { }, new ExponentialBackoffRetryStrategy( retries, Duration.ofMillis(2L), Duration.ofMillis(5L)), - TestingUtils.defaultScheduledExecutor()); + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor())); Boolean result = retryFuture.get(); @@ -336,7 +335,7 @@ public void testScheduleWithInfiniteDelayNeverSchedulesOperation() { FutureUtils.scheduleWithDelay( noOpRunnable, TestingUtils.infiniteTime(), - TestingUtils.defaultScheduledExecutor()); + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor())); assertFalse(completableFuture.isDone()); @@ -360,7 +359,7 @@ public void testOrTimeout() throws Exception { @Test public void testRetryWithDelayRetryStrategyAndPredicate() throws Exception { - final ScheduledExecutorService retryExecutor = TEST_EXECUTOR_RESOURCE.getExecutor(); + final ScheduledExecutorService retryExecutor = EXECUTOR_RESOURCE.getExecutor(); final String retryableExceptionMessage = "first exception"; class TestStringSupplier implements Supplier> { private final AtomicInteger counter = new AtomicInteger(); @@ -675,7 +674,7 @@ public void testSupplyAsyncFailure() throws Exception { () -> { throw testException; }, - TestingUtils.defaultExecutor()); + EXECUTOR_RESOURCE.getExecutor()); try { future.get(); @@ -691,7 +690,7 @@ public void testSupplyAsyncFailure() throws Exception { public void testSupplyAsync() throws Exception { final Object expectedResult = new Object(); final CompletableFuture future = - FutureUtils.supplyAsync(() -> expectedResult, TestingUtils.defaultExecutor()); + FutureUtils.supplyAsync(() -> expectedResult, EXECUTOR_RESOURCE.getExecutor()); assertEquals(future.get(), expectedResult); } @@ -964,7 +963,7 @@ public void testGetWithoutExceptionWithoutFinishing() { public void testSwitchExecutorForNormallyCompletedFuture() { final CompletableFuture source = new CompletableFuture<>(); - final ExecutorService singleThreadExecutor = TEST_EXECUTOR_RESOURCE.getExecutor(); + final ExecutorService singleThreadExecutor = EXECUTOR_RESOURCE.getExecutor(); final CompletableFuture resultFuture = FutureUtils.switchExecutor(source, singleThreadExecutor); @@ -992,7 +991,7 @@ public void testSwitchExecutorForNormallyCompletedFuture() { public void testSwitchExecutorForExceptionallyCompletedFuture() { final CompletableFuture source = new CompletableFuture<>(); - final ExecutorService singleThreadExecutor = TEST_EXECUTOR_RESOURCE.getExecutor(); + final ExecutorService singleThreadExecutor = EXECUTOR_RESOURCE.getExecutor(); final CompletableFuture resultFuture = FutureUtils.switchExecutor(source, singleThreadExecutor); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java index 37d64771e8df3..738d7b8d0510c 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.leaderelection.TestingListener; import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService; import org.apache.flink.runtime.util.TestingFatalErrorHandlerExtension; +import org.apache.flink.testutils.TestingUtils; import org.apache.flink.testutils.executor.TestExecutorExtension; import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.function.RunnableWithException; @@ -40,8 +41,7 @@ import java.util.Collections; import java.util.UUID; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import static org.assertj.core.api.Assertions.assertThat; @@ -58,8 +58,8 @@ public class KubernetesMultipleComponentLeaderElectionDriverTest { new TestingFatalErrorHandlerExtension(); @RegisterExtension - private static final TestExecutorExtension testExecutorExtension = - new TestExecutorExtension<>(Executors::newSingleThreadScheduledExecutor); + private static final TestExecutorExtension testExecutorExtension = + TestingUtils.defaultExecutorExtension(); @Test public void testElectionDriverGainsLeadership() throws Exception { diff --git a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java index 025a1f0571fff..17e892ab90677 100644 --- a/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java +++ b/flink-metrics/flink-metrics-jmx/src/test/java/org/apache/flink/runtime/jobmanager/JMXJobManagerMetricTest.java @@ -39,7 +39,9 @@ import org.apache.flink.test.junit5.InjectClusterClient; import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorExtension; import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -50,6 +52,7 @@ import java.lang.management.ManagementFactory; import java.time.Duration; import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import static org.assertj.core.api.Assertions.assertThat; @@ -57,6 +60,10 @@ /** Tests to verify JMX reporter functionality on the JobManager. */ class JMXJobManagerMetricTest { + @RegisterExtension + static final TestExecutorExtension EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorExtension(); + @RegisterExtension private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = new MiniClusterExtension( @@ -118,7 +125,7 @@ void testJobManagerJMXMetricAccess(@InjectClusterClient ClusterClient client) Time.milliseconds(10), deadline, status -> status == JobStatus.RUNNING, - TestingUtils.defaultScheduledExecutor()) + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor())) .get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); MBeanServer mBeanServer = ManagementFactory.getPlatformMBeanServer(); diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java index 846a86450635a..f1f5af7c8dca5 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java @@ -61,7 +61,6 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.testutils.ClassLoaderUtils; -import org.apache.flink.testutils.TestingUtils; import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.Collector; import org.apache.flink.util.ExceptionUtils; @@ -1347,7 +1346,7 @@ public void close() throws Exception { Time.milliseconds(50), deadline, (jobStatus) -> jobStatus.equals(JobStatus.CANCELED), - TestingUtils.defaultScheduledExecutor()); + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor())); assertEquals( JobStatus.CANCELED, jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlerParameterTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlerParameterTest.java index 4f1b60b2e9170..a22a0f119a88a 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlerParameterTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlerParameterTest.java @@ -33,7 +33,6 @@ import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.runtime.webmonitor.testutils.ParameterProgram; -import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; @@ -53,7 +52,6 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; @@ -92,7 +90,6 @@ enum ProgramArgsParType { CompletableFuture.completedFuture("shazam://localhost:12345"); static Time timeout = Time.seconds(10); static Map responseHeaders = Collections.emptyMap(); - static Executor executor = TestingUtils.defaultExecutor(); private static Path jarWithManifest; private static Path jarWithoutManifest; @@ -129,7 +126,6 @@ static void init() throws Exception { localAddressFuture = CompletableFuture.completedFuture("shazam://localhost:12345"); timeout = Time.seconds(10); responseHeaders = Collections.emptyMap(); - executor = TestingUtils.defaultExecutor(); } @Before diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlerTest.java index 1ca9511e357c1..a3cc743f4aa4e 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlerTest.java @@ -20,6 +20,8 @@ import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -32,6 +34,7 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.util.Optional; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.MatcherAssert.assertThat; @@ -43,6 +46,10 @@ public class JarHandlerTest extends TestLogger { @ClassRule public static final TemporaryFolder TMP = new TemporaryFolder(); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Test public void testPlanJar() throws Exception { runTest("hello out!", "hello err!"); @@ -53,7 +60,9 @@ private static void runTest(String expectedCapturedStdOut, String expectedCaptur final TestingDispatcherGateway restfulGateway = TestingDispatcherGateway.newBuilder().build(); - final JarHandlers handlers = new JarHandlers(TMP.newFolder().toPath(), restfulGateway); + final JarHandlers handlers = + new JarHandlers( + TMP.newFolder().toPath(), restfulGateway, EXECUTOR_RESOURCE.getExecutor()); final Path originalJar = Paths.get(System.getProperty("targetDir")).resolve(JAR_NAME); final Path jar = Files.copy(originalJar, TMP.newFolder().toPath().resolve(JAR_NAME)); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java index 5406edf251af9..6e1fe91f8225f 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarHandlers.java @@ -28,7 +28,6 @@ import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; -import org.apache.flink.testutils.TestingUtils; import java.nio.file.Path; import java.util.Collections; @@ -45,12 +44,14 @@ public class JarHandlers { final JarRunHandler runHandler; final JarDeleteHandler deleteHandler; - JarHandlers(final Path jarDir, final TestingDispatcherGateway restfulGateway) { + JarHandlers( + final Path jarDir, + final TestingDispatcherGateway restfulGateway, + final Executor executor) { final GatewayRetriever gatewayRetriever = () -> CompletableFuture.completedFuture(restfulGateway); final Time timeout = Time.seconds(10); final Map responseHeaders = Collections.emptyMap(); - final Executor executor = TestingUtils.defaultExecutor(); uploadHandler = new JarUploadHandler( diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandlerParameterTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandlerParameterTest.java index 2732790cfeabc..b85e7269959dd 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandlerParameterTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarPlanHandlerParameterTest.java @@ -24,12 +24,16 @@ import org.apache.flink.runtime.rest.handler.HandlerRequest; import org.apache.flink.runtime.rest.messages.JobPlanInfo; import org.apache.flink.runtime.webmonitor.testutils.ParameterProgram; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.junit.BeforeClass; +import org.junit.ClassRule; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; /** Tests for the parameter handling of the {@link JarPlanHandler}. */ @@ -37,6 +41,10 @@ public class JarPlanHandlerParameterTest extends JarHandlerParameterTest { private static JarPlanHandler handler; + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @BeforeClass public static void setup() throws Exception { init(); @@ -48,7 +56,7 @@ public static void setup() throws Exception { JarPlanGetHeaders.getInstance(), jarDir, new Configuration(), - executor, + EXECUTOR_RESOURCE.getExecutor(), jobGraph -> { LAST_SUBMITTED_JOB_GRAPH_REFERENCE.set(jobGraph); return new JobPlanInfo(JsonPlanGenerator.generatePlan(jobGraph)); diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java index 01835b09ad8fa..a0af225e7bb61 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarRunHandlerParameterTest.java @@ -36,12 +36,14 @@ import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.runtime.webmonitor.testutils.ParameterProgram; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; import org.junit.Assert; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import java.nio.file.Files; @@ -54,7 +56,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; import static org.hamcrest.Matchers.containsString; @@ -71,6 +73,10 @@ public class JarRunHandlerParameterTest private static final boolean ALLOW_NON_RESTORED_STATE_QUERY = true; private static final String RESTORE_PATH = "/foo/bar"; + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static JarRunHandler handler; private static Path jarWithEagerSink; @@ -82,7 +88,6 @@ public static void setup() throws Exception { () -> CompletableFuture.completedFuture(restfulGateway); final Time timeout = Time.seconds(10); final Map responseHeaders = Collections.emptyMap(); - final Executor executor = TestingUtils.defaultExecutor(); final Path jarLocation = Paths.get(System.getProperty("targetDir")); final String parameterProgramWithEagerSink = "parameter-program-with-eager-sink.jar"; @@ -99,7 +104,7 @@ public static void setup() throws Exception { JarRunHeaders.getInstance(), jarDir, new Configuration(), - executor, + EXECUTOR_RESOURCE.getExecutor(), ConfigurationVerifyingDetachedApplicationRunner::new); } diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java index fd958572d023f..7a752d64c2745 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/handlers/JarSubmissionITCase.java @@ -22,9 +22,12 @@ import org.apache.flink.runtime.rest.messages.JobPlanInfo; import org.apache.flink.runtime.util.BlobServerResource; import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -33,6 +36,7 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import static org.apache.flink.runtime.webmonitor.handlers.JarHandlers.deleteJar; import static org.apache.flink.runtime.webmonitor.handlers.JarHandlers.listJars; @@ -48,6 +52,10 @@ public class JarSubmissionITCase extends TestLogger { @Rule public final BlobServerResource blobServerResource = new BlobServerResource(); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Test public void testJarSubmission() throws Exception { final TestingDispatcherGateway restfulGateway = @@ -57,7 +65,10 @@ public void testJarSubmission() throws Exception { jobGraph -> CompletableFuture.completedFuture(Acknowledge.get())) .build(); final JarHandlers handlers = - new JarHandlers(temporaryFolder.newFolder().toPath(), restfulGateway); + new JarHandlers( + temporaryFolder.newFolder().toPath(), + restfulGateway, + EXECUTOR_RESOURCE.getExecutor()); final JarUploadHandler uploadHandler = handlers.uploadHandler; final JarListHandler listHandler = handlers.listHandler; final JarPlanHandler planHandler = handlers.planHandler; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java index 36745153959c2..d892319cbd0a7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java @@ -37,16 +37,20 @@ import org.apache.flink.runtime.state.ResultSubpartitionStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.Executors; import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; +import org.junit.ClassRule; import org.junit.Test; import java.util.Collections; import java.util.List; import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicInteger; import static java.util.Collections.emptyList; @@ -65,6 +69,10 @@ /** Tests for failure of checkpoint coordinator. */ public class CheckpointCoordinatorFailureTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + /** * Tests that a failure while storing a completed checkpoint in the completed checkpoint store * will properly fail the originating pending checkpoint and clean upt the completed checkpoint. @@ -79,20 +87,19 @@ public void testFailingCompletedCheckpointStoreAdd() throws Exception { ExecutionGraph testGraph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexId) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex = testGraph.getJobVertex(jobVertexId).getTaskVertices()[0]; // set up the coordinator and validate the initial state CheckpointCoordinator coord = new CheckpointCoordinatorBuilder() - .setExecutionGraph(testGraph) .setCompletedCheckpointStore( new FailingCompletedCheckpointStore( new Exception( "The failing completed checkpoint store failed again... :-("))) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(testGraph); coord.triggerCheckpoint(false); @@ -192,7 +199,7 @@ private void testStoringFailureHandling(Exception failure, int expectedCleanupCa final ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final ExecutionVertex vertex = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; final ExecutionAttemptID attemptId = vertex.getCurrentExecutionAttempt().getAttemptId(); @@ -211,7 +218,6 @@ private void testStoringFailureHandling(Exception failure, int expectedCleanupCa final AtomicInteger cleanupCallCount = new AtomicInteger(0); final CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointIDCounter(checkpointIDCounter) .setCheckpointsCleaner( new CheckpointsCleaner() { @@ -231,7 +237,7 @@ public void cleanCheckpointOnFailedStoring( .setCompletedCheckpointStore(completedCheckpointStore) .setTimer(manuallyTriggeredScheduledExecutor) .setCheckpointStatsTracker(statsTracker) - .build(); + .build(graph); checkpointCoordinator.triggerCheckpoint(false); manuallyTriggeredScheduledExecutor.triggerAll(); CheckpointMetrics expectedReportedMetrics = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java index ce90895f698d5..8e7a5b9720e75 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java @@ -30,10 +30,13 @@ import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.concurrent.Executors; import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.util.concurrent.ScheduledExecutor; +import org.junit.ClassRule; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -47,6 +50,7 @@ import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledExecutorService; import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.StringSerializer; import static org.junit.Assert.assertArrayEquals; @@ -67,6 +71,10 @@ /** Tests for the user-defined hooks that the checkpoint coordinator can call. */ public class CheckpointCoordinatorMasterHooksTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + // ------------------------------------------------------------------------ // hook registration // ------------------------------------------------------------------------ @@ -77,7 +85,7 @@ public void testDeduplicateOnRegister() throws Exception { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(new JobVertexID()) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final CheckpointCoordinator cc = instantiateCheckpointCoordinator(graph); MasterTriggerRestoreHook hook1 = mock(MasterTriggerRestoreHook.class); @@ -100,7 +108,7 @@ public void testNullOrInvalidId() throws Exception { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(new JobVertexID()) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final CheckpointCoordinator cc = instantiateCheckpointCoordinator(graph); try { @@ -139,7 +147,7 @@ public void testHookReset() throws Exception { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(new JobVertexID()) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); CheckpointCoordinator cc = instantiateCheckpointCoordinator(graph); cc.addMasterHook(hook1); @@ -194,7 +202,7 @@ public void testHooksAreCalledOnTrigger() throws Exception { final ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexId) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final ManuallyTriggeredScheduledExecutor manuallyTriggeredScheduledExecutor = new ManuallyTriggeredScheduledExecutor(); final CheckpointCoordinator cc = @@ -301,7 +309,7 @@ public void testHooksAreCalledOnRestore() throws Exception { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(new JobVertexID()) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); CheckpointCoordinator cc = instantiateCheckpointCoordinator(graph); cc.addMasterHook(statefulHook1); @@ -363,7 +371,7 @@ public void checkUnMatchedStateOnRestore() throws Exception { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(new JobVertexID()) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); CheckpointCoordinator cc = instantiateCheckpointCoordinator(graph); cc.addMasterHook(statefulHook); @@ -402,7 +410,7 @@ public void ensureRegisteredAtHookTime() throws Exception { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(new JobVertexID()) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final ManuallyTriggeredScheduledExecutor manuallyTriggeredScheduledExecutor = new ManuallyTriggeredScheduledExecutor(); CheckpointCoordinator cc = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java index 3b839739663fc..2d80c065a9404 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java @@ -39,6 +39,8 @@ import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.types.BooleanValue; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.Executors; @@ -48,6 +50,7 @@ import org.junit.Assert; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -65,6 +68,7 @@ import java.util.Random; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -92,6 +96,10 @@ @SuppressWarnings("checkstyle:EmptyLineSeparator") public class CheckpointCoordinatorRestoringTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; private enum TestScaleType { @@ -136,7 +144,7 @@ private static ExecutionGraph createExecutionGraph(List vertices) builder.addJobVertex( vertex.getId(), vertex.getParallelism(), vertex.getMaxParallelism()); } - return builder.build(); + return builder.build(EXECUTOR_RESOURCE.getExecutor()); } private static class TestingVertex { @@ -197,10 +205,9 @@ public void testRestoreLatestCheckpointedState() throws Exception { // set up the coordinator and validate the initial state final CheckpointCoordinator coordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(executionGraph) .setTimer(manuallyTriggeredScheduledExecutor) .setCompletedCheckpointStore(store) - .build(); + .build(executionGraph); // trigger the checkpoint coordinator.triggerCheckpoint(false); @@ -239,10 +246,9 @@ private void testRestoreLatestCheckpointedState( // set up the coordinator and validate the initial state final CheckpointCoordinator coordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(executionGraph) .setTimer(manuallyTriggeredScheduledExecutor) .setCompletedCheckpointStore(store) - .build(); + .build(executionGraph); final Set executionVertices = vertices.stream() @@ -301,7 +307,7 @@ private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean s new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1, parallelism1, maxParallelism1) .addJobVertex(jobVertexID2, parallelism2, maxParallelism2) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final ExecutionJobVertex jobVertex1 = graph.getJobVertex(jobVertexID1); final ExecutionJobVertex jobVertex2 = graph.getJobVertex(jobVertexID2); @@ -309,10 +315,9 @@ private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean s // set up the coordinator and validate the initial state CheckpointCoordinator coord = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCompletedCheckpointStore(completedCheckpointStore) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); // trigger the checkpoint coord.triggerCheckpoint(false); @@ -415,7 +420,7 @@ private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean s new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1, parallelism1, maxParallelism1) .addJobVertex(jobVertexID2, newParallelism2, maxParallelism2) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final ExecutionJobVertex newJobVertex1 = newGraph.getJobVertex(jobVertexID1); final ExecutionJobVertex newJobVertex2 = newGraph.getJobVertex(jobVertexID2); @@ -423,10 +428,9 @@ private void testRestoreLatestCheckpointedStateWithChangingParallelism(boolean s // set up the coordinator and validate the initial state CheckpointCoordinator newCoord = new CheckpointCoordinatorBuilder() - .setExecutionGraph(newGraph) .setCompletedCheckpointStore(completedCheckpointStore) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(newGraph); Set tasks = new HashSet<>(); tasks.add(newJobVertex1); @@ -506,17 +510,16 @@ public void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1, parallelism1, maxParallelism1) .addJobVertex(jobVertexID2, parallelism2, maxParallelism2) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionJobVertex jobVertex1 = graph.getJobVertex(jobVertexID1); ExecutionJobVertex jobVertex2 = graph.getJobVertex(jobVertexID2); // set up the coordinator and validate the initial state CheckpointCoordinator coord = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCompletedCheckpointStore(completedCheckpointStore) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); // trigger the checkpoint coord.triggerCheckpoint(false); @@ -585,7 +588,7 @@ public void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1, parallelism1, newMaxParallelism1) .addJobVertex(jobVertexID2, parallelism2, newMaxParallelism2) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionJobVertex newJobVertex1 = newGraph.getJobVertex(jobVertexID1); ExecutionJobVertex newJobVertex2 = newGraph.getJobVertex(jobVertexID2); @@ -593,10 +596,9 @@ public void testRestoreLatestCheckpointFailureWhenMaxParallelismChanges() throws // set up the coordinator and validate the initial state CheckpointCoordinator newCoord = new CheckpointCoordinatorBuilder() - .setExecutionGraph(newGraph) .setCompletedCheckpointStore(completedCheckpointStore) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(newGraph); Set tasks = new HashSet<>(); tasks.add(newJobVertex1); @@ -753,7 +755,7 @@ public void testStateRecoveryWithTopologyChange(TestScaleType scaleType) throws .map(OperatorIDPair::generatedIDOnly) .collect(Collectors.toList()), true) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionJobVertex newJobVertex1 = newGraph.getJobVertex(id5.f0); ExecutionJobVertex newJobVertex2 = newGraph.getJobVertex(id3.f0); @@ -781,11 +783,10 @@ public void testStateRecoveryWithTopologyChange(TestScaleType scaleType) throws SharedStateRegistry.DEFAULT_FACTORY.create(Executors.directExecutor(), emptyList()); CheckpointCoordinator coord = new CheckpointCoordinatorBuilder() - .setExecutionGraph(newGraph) .setCompletedCheckpointStore( storeFor(sharedStateRegistry, () -> {}, completedCheckpoint)) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(newGraph); coord.restoreLatestCheckpointedStateToAll(tasks, true); @@ -967,21 +968,20 @@ public void testRestoreLatestCheckpointedStateWithoutInFlightData() throws Excep final ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID, parallelism1, maxParallelism1) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final ExecutionJobVertex jobVertex = graph.getJobVertex(jobVertexID); // set up the coordinator and validate the initial state CheckpointCoordinator coord = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCompletedCheckpointStore(completedCheckpointStore) .setCheckpointCoordinatorConfiguration( new CheckpointCoordinatorConfigurationBuilder() .setCheckpointIdOfIgnoredInFlightData(1) .build()) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); // trigger the checkpoint coord.triggerCheckpoint(false); @@ -1073,7 +1073,7 @@ public void testRestoreFinishedStateWithoutInFlightData() throws Exception { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID, 1, 1, singletonList(op1), true) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); CompletedCheckpointStore completedCheckpointStore = new EmbeddedCompletedCheckpointStore(); Map operatorStates = new HashMap<>(); @@ -1097,13 +1097,12 @@ public void testRestoreFinishedStateWithoutInFlightData() throws Exception { CheckpointCoordinator coord = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration( new CheckpointCoordinatorConfigurationBuilder() .setCheckpointIdOfIgnoredInFlightData(2) .build()) .setCompletedCheckpointStore(completedCheckpointStore) - .build(); + .build(graph); ExecutionJobVertex vertex = graph.getJobVertex(jobVertexID); coord.restoreInitialCheckpointIfPresent(Collections.singleton(vertex)); @@ -1121,7 +1120,7 @@ public void testJobGraphModificationsAreCheckedForInitialCheckpoint() throws Exc ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID, 1, 1) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); CompletedCheckpointStore completedCheckpointStore = new EmbeddedCompletedCheckpointStore(); CompletedCheckpoint completedCheckpoint = new CompletedCheckpoint( @@ -1141,7 +1140,6 @@ public void testJobGraphModificationsAreCheckedForInitialCheckpoint() throws Exc BooleanValue checked = new BooleanValue(false); CheckpointCoordinator restoreCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCompletedCheckpointStore(completedCheckpointStore) .setVertexFinishedStateCheckerFactory( (vertices, states) -> @@ -1151,7 +1149,7 @@ public void validateOperatorsFinishedState() { checked.set(true); } }) - .build(); + .build(graph); restoreCoordinator.restoreInitialCheckpointIfPresent( new HashSet<>(graph.getAllVertices().values())); assertTrue( @@ -1165,12 +1163,11 @@ public void testJobGraphModificationsAreCheckedForSavepoint() throws Exception { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID, 1, 1) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); CheckpointCoordinator coordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); File savepointPath = tmpFolder.newFolder(); CompletableFuture savepointFuture = coordinator.triggerSavepoint( @@ -1192,7 +1189,6 @@ public void testJobGraphModificationsAreCheckedForSavepoint() throws Exception { BooleanValue checked = new BooleanValue(false); CheckpointCoordinator restoreCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setVertexFinishedStateCheckerFactory( (vertices, states) -> new VertexFinishedStateChecker(vertices, states) { @@ -1201,7 +1197,7 @@ public void validateOperatorsFinishedState() { checked.set(true); } }) - .build(); + .build(graph); restoreCoordinator.restoreSavepoint( SavepointRestoreSettings.forPath(savepointFuture.get().getExternalPointer()), graph.getAllVertices(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index e92171de64850..1bdd0e24c461e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -69,6 +69,8 @@ import org.apache.flink.runtime.state.storage.JobManagerCheckpointStorage; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; import org.apache.flink.runtime.testutils.DirectScheduledExecutorService; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; @@ -81,6 +83,7 @@ import org.junit.Assert; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -148,6 +151,10 @@ /** Tests for the checkpoint coordinator. */ public class CheckpointCoordinatorTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Test public void testSharedStateNotDiscaredOnAbort() throws Exception { JobVertexID v1 = new JobVertexID(), v2 = new JobVertexID(); @@ -156,13 +163,12 @@ public void testSharedStateNotDiscaredOnAbort() throws Exception { new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(v1) .addJobVertex(v2) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); CheckpointCoordinator coordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); coordinator.startCheckpointScheduler(); CompletableFuture cpFuture = coordinator.triggerCheckpoint(true); @@ -234,7 +240,7 @@ private void testReportStatsAfterFailure( new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(decliningVertexID) .addJobVertex(lateReportVertexID) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex decliningVertex = executionGraph.getJobVertex(decliningVertexID).getTaskVertices()[0]; @@ -244,10 +250,9 @@ private void testReportStatsAfterFailure( new CheckpointStatsTracker(Integer.MAX_VALUE, new UnregisteredMetricsGroup()); CheckpointCoordinator coordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(executionGraph) .setTimer(manuallyTriggeredScheduledExecutor) .setCheckpointStatsTracker(statsTracker) - .build(); + .build(executionGraph); CompletableFuture result = coordinator.triggerCheckpoint(false); manuallyTriggeredScheduledExecutor.triggerAll(); @@ -364,7 +369,7 @@ public void testMinCheckpointPause() throws Exception { ComponentMainThreadExecutorServiceAdapter .forSingleThreadExecutor( new DirectScheduledExecutorService())) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex = graph.getJobVertex(jobVertexId).getTaskVertices()[0]; ExecutionAttemptID attemptId = vertex.getCurrentExecutionAttempt().getAttemptId(); @@ -379,8 +384,7 @@ public void testMinCheckpointPause() throws Exception { .setMaxConcurrentCheckpoints(1) .setMinPauseBetweenCheckpoints(pause) .build()) - .setExecutionGraph(graph) - .build(); + .build(graph); coordinator.startCheckpointScheduler(); coordinator.triggerCheckpoint( @@ -422,7 +426,7 @@ public void testCheckpointAbortsIfTriggerTasksAreNotExecuted() throws Exception .addJobVertex(new JobVertexID()) .addJobVertex(new JobVertexID(), false) .setTransitToRunning(false) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); CheckpointCoordinator checkpointCoordinator = getCheckpointCoordinator(graph); @@ -452,7 +456,7 @@ public void testCheckpointAbortsIfTriggerTasksAreFinished() throws Exception { new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1) .addJobVertex(jobVertexID2, false) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); CheckpointCoordinator checkpointCoordinator = getCheckpointCoordinator(graph); Arrays.stream(graph.getJobVertex(jobVertexID1).getTaskVertices()) @@ -484,7 +488,7 @@ public void testCheckpointTriggeredAfterSomeTasksFinishedIfAllowed() throws Exce new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1, 3, 256) .addJobVertex(jobVertexID2, 3, 256) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionJobVertex jobVertex1 = graph.getJobVertex(jobVertexID1); ExecutionJobVertex jobVertex2 = graph.getJobVertex(jobVertexID2); @@ -496,11 +500,10 @@ public void testCheckpointTriggeredAfterSomeTasksFinishedIfAllowed() throws Exce new CheckpointStatsTracker(Integer.MAX_VALUE, new UnregisteredMetricsGroup()); CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setTimer(manuallyTriggeredScheduledExecutor) .setAllowCheckpointsAfterTasksFinished(true) .setCheckpointStatsTracker(statsTracker) - .build(); + .build(graph); // nothing should be happening assertEquals(0, checkpointCoordinator.getNumberOfPendingCheckpoints()); @@ -547,7 +550,7 @@ public void testTasksFinishDuringTriggering() throws Exception { .setTransitToRunning(false) .addJobVertex(jobVertexID1, 1, 256) .addJobVertex(jobVertexID2, 1, 256) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionJobVertex jobVertex1 = graph.getJobVertex(jobVertexID1); ExecutionVertex taskVertex = jobVertex1.getTaskVertices()[0]; ExecutionJobVertex jobVertex2 = graph.getJobVertex(jobVertexID2); @@ -593,10 +596,9 @@ public void notifyCheckpointAborted( CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setTimer(manuallyTriggeredScheduledExecutor) .setAllowCheckpointsAfterTasksFinished(true) - .build(); + .build(graph); // nothing should be happening assertEquals(0, checkpointCoordinator.getNumberOfPendingCheckpoints()); @@ -620,7 +622,7 @@ public void testTriggerAndDeclineCheckpointThenFailureManagerThrowsException() new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1) .addJobVertex(jobVertexID2) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; ExecutionVertex vertex2 = graph.getJobVertex(jobVertexID2).getTaskVertices()[0]; final ExecutionAttemptID attemptID1 = vertex1.getCurrentExecutionAttempt().getAttemptId(); @@ -682,7 +684,7 @@ public void testIOExceptionCheckpointExceedsTolerableFailureNumber() throws Exce new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(new JobVertexID()) .addJobVertex(new JobVertexID()) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final String expectedErrorMessage = "Expected Error Message"; CheckpointFailureManager checkpointFailureManager = @@ -744,7 +746,7 @@ public void testTriggerCheckpointAfterCheckpointStorageIOException() throws Exce .setFailureManager(new CheckpointFailureManager(0, failureCallback)) .setCheckpointStorage(new IOExceptionCheckpointStorage()) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); // when: The checkpoint is triggered. testTriggerCheckpoint(checkpointCoordinator, IO_EXCEPTION); @@ -764,15 +766,14 @@ public void testCheckpointAbortsIfTriggerTasksAreFinishedAndIOException() throws new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1) .addJobVertex(jobVertexID2, false) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); // set up the coordinator CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointStorage(new IOExceptionCheckpointStorage()) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); Arrays.stream(graph.getJobVertex(jobVertexID1).getTaskVertices()) .forEach(task -> task.getCurrentExecutionAttempt().markFinished()); @@ -803,7 +804,7 @@ public void testExpiredCheckpointExceedsTolerableFailureNumber() throws Exceptio new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(new JobVertexID()) .addJobVertex(new JobVertexID()) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final String errorMsg = "Exceeded checkpoint failure tolerance number!"; CheckpointFailureManager checkpointFailureManager = getCheckpointFailureManager(errorMsg); @@ -856,7 +857,7 @@ private void testTriggerAndDeclineCheckpointSimple( .addJobVertex(jobVertexID1) .addJobVertex(jobVertexID2) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; ExecutionVertex vertex2 = graph.getJobVertex(jobVertexID2).getTaskVertices()[0]; @@ -868,7 +869,6 @@ private void testTriggerAndDeclineCheckpointSimple( // set up the coordinator and validate the initial state CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration( CheckpointCoordinatorConfiguration.builder() .setAlignedCheckpointTimeout(Long.MAX_VALUE) @@ -877,7 +877,7 @@ private void testTriggerAndDeclineCheckpointSimple( .setTimer(manuallyTriggeredScheduledExecutor) .setCheckpointFailureManager( new CheckpointFailureManager(0, failJobCallback)) - .build(); + .build(graph); assertEquals(0, checkpointCoordinator.getNumberOfPendingCheckpoints()); assertEquals(0, checkpointCoordinator.getNumberOfRetainedSuccessfulCheckpoints()); @@ -986,7 +986,7 @@ public void testTriggerAndDeclineCheckpointComplex() throws Exception { .addJobVertex(jobVertexID1) .addJobVertex(jobVertexID2) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; ExecutionVertex vertex2 = graph.getJobVertex(jobVertexID2).getTaskVertices()[0]; @@ -1136,7 +1136,7 @@ public void testTriggerAndConfirmSimpleCheckpoint() throws Exception { .addJobVertex(jobVertexID1) .addJobVertex(jobVertexID2) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; ExecutionVertex vertex2 = graph.getJobVertex(jobVertexID2).getTaskVertices()[0]; @@ -1310,7 +1310,7 @@ public void testMultipleConcurrentCheckpoints() throws Exception { .addJobVertex(jobVertexID2) .addJobVertex(jobVertexID3, false) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; ExecutionVertex vertex2 = graph.getJobVertex(jobVertexID2).getTaskVertices()[0]; @@ -1323,14 +1323,13 @@ public void testMultipleConcurrentCheckpoints() throws Exception { // set up the coordinator and validate the initial state CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration( CheckpointCoordinatorConfiguration.builder() .setMaxConcurrentCheckpoints(Integer.MAX_VALUE) .build()) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); assertEquals(0, checkpointCoordinator.getNumberOfPendingCheckpoints()); assertEquals(0, checkpointCoordinator.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1466,7 +1465,7 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() throws Exception { .addJobVertex(jobVertexID2) .addJobVertex(jobVertexID3, false) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; ExecutionVertex vertex2 = graph.getJobVertex(jobVertexID2).getTaskVertices()[0]; @@ -1481,14 +1480,13 @@ public void testSuccessfulCheckpointSubsumesUnsuccessful() throws Exception { new StandaloneCompletedCheckpointStore(10); CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration( CheckpointCoordinatorConfiguration.builder() .setMaxConcurrentCheckpoints(Integer.MAX_VALUE) .build()) .setCompletedCheckpointStore(completedCheckpointStore) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); assertEquals(0, checkpointCoordinator.getNumberOfPendingCheckpoints()); assertEquals(0, checkpointCoordinator.getNumberOfRetainedSuccessfulCheckpoints()); @@ -1680,7 +1678,7 @@ public void testCheckpointTimeoutIsolated() throws Exception { .addJobVertex(jobVertexID1) .addJobVertex(jobVertexID2, false) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; ExecutionVertex vertex2 = graph.getJobVertex(jobVertexID2).getTaskVertices()[0]; @@ -1690,10 +1688,9 @@ public void testCheckpointTimeoutIsolated() throws Exception { // set up the coordinator CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); // trigger a checkpoint, partially acknowledged final CompletableFuture checkpointFuture = @@ -1753,7 +1750,7 @@ public void testHandleMessagesForNonExistingCheckpoints() throws Exception { .addJobVertex(jobVertexID1) .addJobVertex(jobVertexID2, false) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; @@ -1761,10 +1758,9 @@ public void testHandleMessagesForNonExistingCheckpoints() throws Exception { CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); final CompletableFuture checkpointFuture = checkpointCoordinator.triggerCheckpoint(false); @@ -1817,7 +1813,7 @@ public void testStateCleanupForLateOrUnknownMessages() throws Exception { .addJobVertex(jobVertexID1) .addJobVertex(jobVertexID2, false) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; ExecutionVertex vertex2 = graph.getJobVertex(jobVertexID2).getTaskVertices()[0]; @@ -1831,10 +1827,9 @@ public void testStateCleanupForLateOrUnknownMessages() throws Exception { .build(); CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration(chkConfig) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); final CompletableFuture checkpointFuture = checkpointCoordinator.triggerCheckpoint(false); @@ -2001,7 +1996,7 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { .addJobVertex(jobVertexID1) .addJobVertex(jobVertexID2) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; ExecutionVertex vertex2 = graph.getJobVertex(jobVertexID2).getTaskVertices()[0]; @@ -2013,7 +2008,6 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { // set up the coordinator and validate the initial state CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration( CheckpointCoordinatorConfiguration.builder() .setAlignedCheckpointTimeout(Long.MAX_VALUE) @@ -2021,7 +2015,7 @@ public void testTriggerAndConfirmSimpleSavepoint() throws Exception { .build()) .setTimer(manuallyTriggeredScheduledExecutor) .setCheckpointStatsTracker(statsTracker) - .build(); + .build(graph); assertEquals(0, checkpointCoordinator.getNumberOfPendingCheckpoints()); assertEquals(0, checkpointCoordinator.getNumberOfRetainedSuccessfulCheckpoints()); @@ -2138,7 +2132,7 @@ public void testSavepointsAreNotSubsumed() throws Exception { new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1) .addJobVertex(jobVertexID2) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; ExecutionVertex vertex2 = graph.getJobVertex(jobVertexID2).getTaskVertices()[0]; @@ -2152,7 +2146,6 @@ public void testSavepointsAreNotSubsumed() throws Exception { CheckpointCoordinator checkpointCoordinator = spy( new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration( CheckpointCoordinatorConfiguration.builder() .setMaxConcurrentCheckpoints(Integer.MAX_VALUE) @@ -2161,7 +2154,7 @@ public void testSavepointsAreNotSubsumed() throws Exception { .setCompletedCheckpointStore( new StandaloneCompletedCheckpointStore(1)) .setTimer(manuallyTriggeredScheduledExecutor) - .build()); + .build(graph)); String savepointDir = tmpFolder.newFolder().getAbsolutePath(); @@ -2283,7 +2276,7 @@ private void testMaxConcurrentAttempts(int maxConcurrentAttempts) { new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; @@ -2299,11 +2292,10 @@ private void testMaxConcurrentAttempts(int maxConcurrentAttempts) { .build(); CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration(chkConfig) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); checkpointCoordinator.startCheckpointScheduler(); @@ -2352,7 +2344,7 @@ public void testMaxConcurrentAttemptsWithSubsumption() throws Exception { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; @@ -2367,11 +2359,10 @@ public void testMaxConcurrentAttemptsWithSubsumption() throws Exception { .build(); CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration(chkConfig) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); checkpointCoordinator.startCheckpointScheduler(); @@ -2426,7 +2417,7 @@ private CheckpointCoordinator setupCheckpointCoordinatorWithInactiveTasks( new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1) .setTransitToRunning(false) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; @@ -2440,13 +2431,12 @@ private CheckpointCoordinator setupCheckpointCoordinatorWithInactiveTasks( CheckpointIDCounterWithOwner checkpointIDCounter = new CheckpointIDCounterWithOwner(); CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration(chkConfig) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) .setCheckpointStorage(checkpointStorage) .setTimer(manuallyTriggeredScheduledExecutor) .setCheckpointIDCounter(checkpointIDCounter) - .build(); + .build(graph); checkpointIDCounter.setOwner(checkpointCoordinator); checkpointCoordinator.startCheckpointScheduler(); @@ -2476,7 +2466,7 @@ public void testConcurrentSavepoints() throws Exception { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; @@ -2491,12 +2481,11 @@ public void testConcurrentSavepoints() throws Exception { .build(); CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration(chkConfig) .setCheckpointIDCounter(checkpointIDCounter) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); List> savepointFutures = new ArrayList<>(); @@ -2544,7 +2533,7 @@ public void testMinDelayBetweenSavepoints() throws Exception { .setCheckpointCoordinatorConfiguration(chkConfig) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); String savepointDir = tmpFolder.newFolder().getAbsolutePath(); @@ -2563,7 +2552,7 @@ public void testExternalizedCheckpoints() throws Exception { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(new JobVertexID()) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); // set up the coordinator and validate the initial state CheckpointCoordinatorConfiguration chkConfig = @@ -2572,10 +2561,9 @@ public void testExternalizedCheckpoints() throws Exception { .build(); CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration(chkConfig) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); CompletableFuture checkpointFuture = checkpointCoordinator.triggerCheckpoint(false); @@ -2812,7 +2800,7 @@ public void testCheckpointStatsTrackerPendingCheckpointCallback() throws Excepti new CheckpointCoordinatorBuilder() .setTimer(manuallyTriggeredScheduledExecutor) .setCheckpointStatsTracker(tracker) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); when(tracker.reportPendingCheckpoint( anyLong(), anyLong(), any(CheckpointProperties.class), any(Map.class))) @@ -2846,7 +2834,7 @@ public void testCheckpointStatsTrackerRestoreCallback() throws Exception { .setCompletedCheckpointStore(store) .setTimer(manuallyTriggeredScheduledExecutor) .setCheckpointStatsTracker(tracker) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); store.addCheckpointAndSubsumeOldestOne( new CompletedCheckpoint( @@ -2880,7 +2868,7 @@ public void testSharedStateRegistrationOnRestore() throws Exception { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1, parallelism1, maxParallelism1) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionJobVertex jobVertex1 = graph.getJobVertex(jobVertexID1); @@ -2893,11 +2881,9 @@ public void testSharedStateRegistrationOnRestore() throws Exception { // set up the coordinator and validate the initial state final CheckpointCoordinatorBuilder coordinatorBuilder = - new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) - .setTimer(manuallyTriggeredScheduledExecutor); + new CheckpointCoordinatorBuilder().setTimer(manuallyTriggeredScheduledExecutor); final CheckpointCoordinator coordinator = - coordinatorBuilder.setCompletedCheckpointStore(store).build(); + coordinatorBuilder.setCompletedCheckpointStore(store).build(graph); final int numCheckpoints = 3; @@ -2989,7 +2975,7 @@ public void testSharedStateRegistrationOnRestore() throws Exception { final EmbeddedCompletedCheckpointStore secondStore = new EmbeddedCompletedCheckpointStore(10, store.getAllCheckpoints(), secondInstance); final CheckpointCoordinator secondCoordinator = - coordinatorBuilder.setCompletedCheckpointStore(secondStore).build(); + coordinatorBuilder.setCompletedCheckpointStore(secondStore).build(graph); assertTrue(secondCoordinator.restoreLatestCheckpointedStateToAll(tasks, false)); // validate that all shared states are registered again after the recovery. @@ -3043,7 +3029,7 @@ public void jobFailsIfInFlightSynchronousSavepointIsDiscarded() throws Exception new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1) .addJobVertex(jobVertexID2) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; ExecutionVertex vertex2 = graph.getJobVertex(jobVertexID2).getTaskVertices()[0]; @@ -3113,7 +3099,7 @@ public void testTriggerCheckpointAfterStopping() throws Exception { new CheckpointCoordinatorBuilder() .setCheckpointIDCounter(testingCounter) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); testingCounter.setOwner(checkpointCoordinator); testTriggerCheckpoint(checkpointCoordinator, PERIODIC_SCHEDULER_SHUTDOWN); @@ -3135,7 +3121,7 @@ public void testTriggerCheckpointWithCounterIOException() throws Exception { .setFailureManager(new CheckpointFailureManager(0, failureCallback)) .setTimer(manuallyTriggeredScheduledExecutor) .setCheckpointStatsTracker(statsTracker) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); testingCounter.setOwner(checkpointCoordinator); // when: The checkpoint is triggered. @@ -3196,7 +3182,7 @@ public void testSavepointScheduledInUnalignedMode() throws Exception { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(new JobVertexID()) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); CheckpointCoordinator coordinator = new CheckpointCoordinatorBuilder() .setCheckpointCoordinatorConfiguration( @@ -3204,9 +3190,8 @@ public void testSavepointScheduledInUnalignedMode() throws Exception { .setUnalignedCheckpointsEnabled(true) .setMaxConcurrentCheckpoints(maxConcurrentCheckpoints) .build()) - .setExecutionGraph(graph) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); try { List> checkpointFutures = new ArrayList<>(checkpointRequestsToSend); coordinator.startCheckpointScheduler(); @@ -3267,7 +3252,7 @@ public void testExternallyInducedSourceWithOperatorCoordinator() throws Exceptio .addJobVertex(jobVertexID1) .addJobVertex(jobVertexID2) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; ExecutionVertex vertex2 = graph.getJobVertex(jobVertexID2).getTaskVertices()[0]; @@ -3301,7 +3286,6 @@ public void testExternallyInducedSourceWithOperatorCoordinator() throws Exceptio // set up the coordinator and validate the initial state CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration( CheckpointCoordinatorConfiguration.builder() .setMaxConcurrentCheckpoints(Integer.MAX_VALUE) @@ -3309,7 +3293,7 @@ public void testExternallyInducedSourceWithOperatorCoordinator() throws Exceptio .setTimer(manuallyTriggeredScheduledExecutor) .setCoordinatorsToCheckpoint( Collections.singleton(coordinatorCheckpointContext)) - .build(); + .build(graph); AtomicReference checkpointIdRef = new AtomicReference<>(); // Add a master hook which triggers and acks the task checkpoint immediately. @@ -3421,7 +3405,7 @@ public void testCompleteCheckpointFailureWithExternallyInducedSource() throws Ex new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID1) .addJobVertex(jobVertexID2) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex1 = graph.getJobVertex(jobVertexID1).getTaskVertices()[0]; ExecutionVertex vertex2 = graph.getJobVertex(jobVertexID2).getTaskVertices()[0]; @@ -3455,7 +3439,6 @@ public void testCompleteCheckpointFailureWithExternallyInducedSource() throws Ex // set up the coordinator and validate the initial state CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration( CheckpointCoordinatorConfiguration.builder() .setMaxConcurrentCheckpoints(Integer.MAX_VALUE) @@ -3492,7 +3475,7 @@ public CheckpointStorageAccess createCheckpointStorage( }; } }) - .build(); + .build(graph); AtomicReference checkpointIdRef = new AtomicReference<>(); // Add a master hook which triggers and acks the task checkpoint immediately. @@ -3577,7 +3560,7 @@ public void testResetCalledInRegionRecovery() throws Exception { CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); TestResetHook hook = new TestResetHook("id"); @@ -3593,7 +3576,7 @@ public void testNotifyCheckpointAbortionInOperatorCoordinator() throws Exception ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex executionVertex = graph.getJobVertex(jobVertexID).getTaskVertices()[0]; ExecutionAttemptID attemptID = executionVertex.getCurrentExecutionAttempt().getAttemptId(); @@ -3609,14 +3592,13 @@ public void testNotifyCheckpointAbortionInOperatorCoordinator() throws Exception // set up the coordinator and validate the initial state CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration( CheckpointCoordinatorConfiguration.builder() .setMaxConcurrentCheckpoints(Integer.MAX_VALUE) .build()) .setTimer(manuallyTriggeredScheduledExecutor) .setCoordinatorsToCheckpoint(Collections.singleton(context)) - .build(); + .build(graph); try { // Trigger checkpoint 1. checkpointCoordinator.triggerCheckpoint(false); @@ -3698,7 +3680,7 @@ public void testReportLatestCompletedCheckpointIdWithAbort() throws Exception { new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID) .setTransitToRunning(false) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex task = graph.getJobVertex(jobVertexID).getTaskVertices()[0]; @@ -3723,10 +3705,9 @@ public void notifyCheckpointAborted( CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setTimer(manuallyTriggeredScheduledExecutor) .setAllowCheckpointsAfterTasksFinished(true) - .build(); + .build(graph); // Trigger a successful checkpoint CompletableFuture result = @@ -3769,16 +3750,15 @@ public void testBaseLocationsNotInitialized() throws Exception { new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID) .setTransitToRunning(false) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration( CheckpointCoordinatorConfiguration.builder() .setCheckpointInterval(Long.MAX_VALUE) .build()) .setCheckpointStorage(new FsStateBackend(checkpointDir.toURI())) - .build(); + .build(graph); Path jobCheckpointPath = new Path(checkpointDir.getAbsolutePath(), graph.getJobID().toString()); FileSystem fs = FileSystem.get(checkpointDir.toURI()); @@ -3789,24 +3769,22 @@ public void testBaseLocationsNotInitialized() throws Exception { private CheckpointCoordinator getCheckpointCoordinator(ExecutionGraph graph) throws Exception { return new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration( CheckpointCoordinatorConfiguration.builder() .setAlignedCheckpointTimeout(Long.MAX_VALUE) .setMaxConcurrentCheckpoints(Integer.MAX_VALUE) .build()) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); } private CheckpointCoordinator getCheckpointCoordinator( ExecutionGraph graph, CheckpointFailureManager failureManager) throws Exception { return new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setTimer(manuallyTriggeredScheduledExecutor) .setFailureManager(failureManager) - .build(); + .build(graph); } private CheckpointCoordinator getCheckpointCoordinator(ScheduledExecutor timer) @@ -3815,10 +3793,10 @@ private CheckpointCoordinator getCheckpointCoordinator(ScheduledExecutor timer) new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(new JobVertexID()) .addJobVertex(new JobVertexID()) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); // set up the coordinator and validate the initial state - return new CheckpointCoordinatorBuilder().setExecutionGraph(graph).setTimer(timer).build(); + return new CheckpointCoordinatorBuilder().setTimer(timer).build(graph); } private CheckpointFailureManager getCheckpointFailureManager(String errorMsg) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java index 170b9899587c6..8112e2ebc9abc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTestingUtils.java @@ -84,6 +84,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledExecutorService; import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.Consumer; @@ -652,7 +653,7 @@ public CheckpointExecutionGraphBuilder setMainThreadExecutor( return this; } - ExecutionGraph build() throws Exception { + ExecutionGraph build(ScheduledExecutorService executorService) throws Exception { // Lets connect source vertices and non-source vertices for (JobVertex source : sourceVertices) { for (JobVertex nonSource : nonSourceVertices) { @@ -666,8 +667,8 @@ ExecutionGraph build() throws Exception { allVertices.addAll(nonSourceVertices); ExecutionGraph executionGraph = - ExecutionGraphTestUtils.createSimpleTestGraph( - allVertices.toArray(new JobVertex[0])); + ExecutionGraphTestUtils.createExecutionGraph( + executorService, allVertices.toArray(new JobVertex[0])); executionGraph.start(mainThreadExecutor); if (taskManagerGateway != null) { @@ -704,8 +705,6 @@ public static class CheckpointCoordinatorBuilder { .setMaxConcurrentCheckpoints(Integer.MAX_VALUE) .build(); - private ExecutionGraph executionGraph; - private Collection coordinatorsToCheckpoint = Collections.emptyList(); @@ -742,11 +741,6 @@ public CheckpointCoordinatorBuilder setCheckpointCoordinatorConfiguration( return this; } - public CheckpointCoordinatorBuilder setExecutionGraph(ExecutionGraph executionGraph) { - this.executionGraph = executionGraph; - return this; - } - public CheckpointCoordinatorBuilder setCoordinatorsToCheckpoint( Collection coordinatorsToCheckpoint) { this.coordinatorsToCheckpoint = coordinatorsToCheckpoint; @@ -820,13 +814,15 @@ public CheckpointCoordinatorBuilder setVertexFinishedStateCheckerFactory( return this; } - public CheckpointCoordinator build() throws Exception { - if (executionGraph == null) { - executionGraph = - new CheckpointExecutionGraphBuilder() - .addJobVertex(new JobVertexID()) - .build(); - } + public CheckpointCoordinator build(ScheduledExecutorService executorService) + throws Exception { + return build( + new CheckpointExecutionGraphBuilder() + .addJobVertex(new JobVertexID()) + .build(executorService)); + } + + public CheckpointCoordinator build(ExecutionGraph executionGraph) throws Exception { DefaultCheckpointPlanCalculator checkpointPlanCalculator = new DefaultCheckpointPlanCalculator( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java index 643c09d95c684..3eb7170c0c636 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java @@ -32,6 +32,8 @@ import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration.CheckpointCoordinatorConfigurationBuilder; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.TestLogger; @@ -40,6 +42,7 @@ import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -68,6 +71,11 @@ /** Tests for checkpoint coordinator triggering. */ public class CheckpointCoordinatorTriggeringTest extends TestLogger { + + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; private ManuallyTriggeredScheduledExecutor manuallyTriggeredScheduledExecutor; @@ -92,7 +100,7 @@ public void testPeriodicTriggering() { new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex = graph.getJobVertex(jobVertexID).getTaskVertices()[0]; ExecutionAttemptID attemptID = vertex.getCurrentExecutionAttempt().getAttemptId(); @@ -105,12 +113,11 @@ public void testPeriodicTriggering() { .build(); CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration( checkpointCoordinatorConfiguration) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); checkpointCoordinator.startCheckpointScheduler(); @@ -186,7 +193,7 @@ public void testTriggeringFullSnapshotAfterJobmasterFailover() throws Exception new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex = graph.getJobVertex(jobVertexID).getTaskVertices()[0]; ExecutionAttemptID attemptID = vertex.getCurrentExecutionAttempt().getAttemptId(); @@ -238,7 +245,7 @@ public void testTriggeringFullCheckpoints() throws Exception { new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex = graph.getJobVertex(jobVertexID).getTaskVertices()[0]; ExecutionAttemptID attemptID = vertex.getCurrentExecutionAttempt().getAttemptId(); @@ -320,12 +327,11 @@ private CheckpointCoordinator createCheckpointCoordinator( .setMaxConcurrentCheckpoints(Integer.MAX_VALUE) .build(); return new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration(checkpointCoordinatorConfiguration) .setCompletedCheckpointStore(checkpointStore) .setCheckpointIDCounter(checkpointIDCounter) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); } /** @@ -343,7 +349,7 @@ public void testMinTimeBetweenCheckpointsInterval() throws Exception { new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex = graph.getJobVertex(jobVertexID).getTaskVertices()[0]; ExecutionAttemptID attemptID = vertex.getCurrentExecutionAttempt().getAttemptId(); @@ -360,11 +366,10 @@ public void testMinTimeBetweenCheckpointsInterval() throws Exception { .build(); final CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointCoordinatorConfiguration(checkpointCoordinatorConfiguration) .setCompletedCheckpointStore(new StandaloneCompletedCheckpointStore(2)) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); try { checkpointCoordinator.startCheckpointScheduler(); @@ -477,7 +482,7 @@ public void testTriggerCheckpointBeforePreviousOneCompleted() throws Exception { new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex = graph.getJobVertex(jobVertexID).getTaskVertices()[0]; ExecutionAttemptID attemptID = vertex.getCurrentExecutionAttempt().getAttemptId(); @@ -517,7 +522,7 @@ public void testTriggerCheckpointRequestQueuedWithFailure() throws Exception { new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex = graph.getJobVertex(jobVertexID).getTaskVertices()[0]; ExecutionAttemptID attemptID = vertex.getCurrentExecutionAttempt().getAttemptId(); @@ -525,10 +530,9 @@ public void testTriggerCheckpointRequestQueuedWithFailure() throws Exception { // set up the coordinator and validate the initial state CheckpointCoordinator checkpointCoordinator = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setCheckpointIDCounter(new UnstableCheckpointIDCounter(id -> id == 0)) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); checkpointCoordinator.startCheckpointScheduler(); // start a periodic checkpoint first @@ -568,7 +572,7 @@ public void testTriggerCheckpointRequestCancelled() throws Exception { new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex = graph.getJobVertex(jobVertexID).getTaskVertices()[0]; ExecutionAttemptID attemptID = vertex.getCurrentExecutionAttempt().getAttemptId(); @@ -619,7 +623,7 @@ public void testTriggerCheckpointInitializationFailed() throws Exception { new CheckpointCoordinatorBuilder() .setCheckpointIDCounter(new UnstableCheckpointIDCounter(id -> id == 0)) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); checkpointCoordinator.startCheckpointScheduler(); final CompletableFuture onCompletionPromise1 = @@ -662,7 +666,7 @@ public void testTriggerCheckpointSnapshotMasterHookFailed() throws Exception { new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID) .setTaskManagerGateway(gateway) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex vertex = graph.getJobVertex(jobVertexID).getTaskVertices()[0]; ExecutionAttemptID attemptID = vertex.getCurrentExecutionAttempt().getAttemptId(); @@ -712,7 +716,7 @@ public void discardingTriggeringCheckpointWillExecuteNextCheckpointRequest() thr .setTimer(new ScheduledExecutorServiceAdapter(scheduledExecutorService)) .setCheckpointCoordinatorConfiguration( CheckpointCoordinatorConfiguration.builder().build()) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final CompletableFuture masterHookCheckpointFuture = new CompletableFuture<>(); final OneShotLatch triggerCheckpointLatch = new OneShotLatch(); @@ -750,15 +754,14 @@ public void discardingTriggeringCheckpointWillExecuteNextCheckpointRequest() thr private CheckpointCoordinator createCheckpointCoordinator() throws Exception { return new CheckpointCoordinatorBuilder() .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); } private CheckpointCoordinator createCheckpointCoordinator(ExecutionGraph graph) throws Exception { return new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); } private CompletableFuture triggerPeriodicCheckpoint( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java index fb15d31cf2945..7fad3f06d7bdd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointSettingsSerializableTest.java @@ -42,10 +42,13 @@ import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.ttl.TtlTimeProvider; import org.apache.flink.testutils.ClassLoaderUtils; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TernaryBoolean; import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; import org.junit.Test; import javax.annotation.Nonnull; @@ -53,6 +56,7 @@ import java.io.IOException; import java.io.Serializable; import java.util.Collection; +import java.util.concurrent.ScheduledExecutorService; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -64,6 +68,9 @@ * user-defined objects. */ public class CheckpointSettingsSerializableTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); @Test public void testDeserializationOfUserCodeWithUserClassLoader() throws Exception { @@ -107,7 +114,7 @@ public void testDeserializationOfUserCodeWithUserClassLoader() throws Exception TestingDefaultExecutionGraphBuilder.newBuilder() .setJobGraph(copy) .setUserClassLoader(classLoader) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); assertEquals(1, eg.getCheckpointCoordinator().getNumberOfRegisteredMasterHooks()); assertTrue( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java index 62398922c2ea5..3bd79e3ca0466 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStateRestoreTest.java @@ -34,9 +34,12 @@ import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyedStateHandle; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.SerializableObject; import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; +import org.junit.ClassRule; import org.junit.Test; import java.util.Arrays; @@ -46,6 +49,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -58,6 +62,10 @@ /** Tests concerning the restoring of state from a checkpoint to the task executions. */ public class CheckpointStateRestoreTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static final String TASK_MANAGER_LOCATION_INFO = "Unknown location"; /** Tests that on restore the task state is reset for each stateful task. */ @@ -78,7 +86,7 @@ public void testSetState() { new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(statefulId, 3, 256) .addJobVertex(statelessId, 2, 256) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionJobVertex stateful = graph.getJobVertex(statefulId); ExecutionJobVertex stateless = graph.getJobVertex(statelessId); @@ -100,9 +108,8 @@ public void testSetState() { CheckpointCoordinator coord = new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) .setTimer(manuallyTriggeredScheduledExecutor) - .build(); + .build(graph); // create ourselves a checkpoint with state coord.triggerCheckpoint(false); @@ -175,7 +182,8 @@ public void testSetState() { @Test public void testNoCheckpointAvailable() { try { - CheckpointCoordinator coord = new CheckpointCoordinatorBuilder().build(); + CheckpointCoordinator coord = + new CheckpointCoordinatorBuilder().build(EXECUTOR_RESOURCE.getExecutor()); final boolean restored = coord.restoreLatestCheckpointedStateToAll(Collections.emptySet(), false); @@ -217,7 +225,8 @@ public void testNonRestoredState() throws Exception { tasks.add(jobVertex1); tasks.add(jobVertex2); - CheckpointCoordinator coord = new CheckpointCoordinatorBuilder().build(); + CheckpointCoordinator coord = + new CheckpointCoordinatorBuilder().build(EXECUTOR_RESOURCE.getExecutor()); // --- (2) Checkpoint misses state for a jobVertex (should work) --- Map checkpointTaskStates = new HashMap<>(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java index 84ebe3a3c71d6..123a71effe146 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTrackerTest.java @@ -25,7 +25,10 @@ import org.apache.flink.runtime.executiongraph.ExecutionGraph; import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; +import org.junit.ClassRule; import org.junit.Test; import java.util.ArrayList; @@ -34,6 +37,7 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; import static java.util.Collections.singletonMap; import static org.junit.Assert.assertEquals; @@ -44,6 +48,10 @@ public class CheckpointStatsTrackerTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + /** Tests that the number of remembered checkpoints configuration is respected. */ @Test public void testTrackerWithoutHistory() throws Exception { @@ -51,7 +59,7 @@ public void testTrackerWithoutHistory() throws Exception { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID, 3, 256) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionJobVertex jobVertex = graph.getJobVertex(jobVertexID); CheckpointStatsTracker tracker = @@ -97,7 +105,7 @@ public void testCheckpointTracking() throws Exception { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID, 3, 256) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionJobVertex jobVertex = graph.getJobVertex(jobVertexID); Map vertexToDop = singletonMap(jobVertexID, jobVertex.getParallelism()); @@ -331,7 +339,7 @@ public > G gauge(String name, G gauge) { ExecutionGraph graph = new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionJobVertex jobVertex = graph.getJobVertex(jobVertexID); CheckpointStatsTracker stats = new CheckpointStatsTracker(0, metricGroup); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanCalculatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanCalculatorTest.java index 485e97a59655f..c23d8580efb43 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanCalculatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanCalculatorTest.java @@ -32,8 +32,11 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.hamcrest.CoreMatchers; +import org.junit.ClassRule; import org.junit.Test; import java.util.ArrayList; @@ -46,6 +49,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -64,6 +68,10 @@ */ public class DefaultCheckpointPlanCalculatorTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Test public void testComputeAllRunningGraph() throws Exception { runSingleTest( @@ -172,7 +180,7 @@ private void runWithNotRunningTask( .addJobVertex(runningVertex, isRunningVertexSource) .addJobVertex(notRunningVertex, isNotRunningVertexSource) .setTransitToRunning(false) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); // The first vertex is always RUNNING. transitVertexToState(graph, runningVertex, ExecutionState.RUNNING); @@ -298,7 +306,9 @@ private ExecutionGraph createExecutionGraph( ResultPartitionType.PIPELINED); } - ExecutionGraph graph = ExecutionGraphTestUtils.createSimpleTestGraph(jobVertices); + ExecutionGraph graph = + ExecutionGraphTestUtils.createExecutionGraph( + EXECUTOR_RESOURCE.getExecutor(), jobVertices); graph.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); graph.transitionToRunning(); graph.getAllExecutionVertices() diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanTest.java index b32d075fd560c..600b8b48d979e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointPlanTest.java @@ -26,6 +26,8 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.TestingStreamStateHandle; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.FlinkRuntimeException; import org.junit.ClassRule; @@ -38,6 +40,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ScheduledExecutorService; import static org.apache.flink.runtime.checkpoint.CheckpointCoordinatorTestingUtils.createSubtaskStateWithUnionListState; import static org.junit.Assert.assertEquals; @@ -47,6 +50,10 @@ /** Tests the behavior of the {@link DefaultCheckpointPlan}. */ public class DefaultCheckpointPlanTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @Rule public final ExpectedException expectedException = ExpectedException.none(); @@ -65,7 +72,7 @@ public void testAbortionIfPartiallyFinishedVertexUsedUnionListState() throws Exc Collections.singletonList( OperatorIDPair.generatedIDOnly(operatorId)), true) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex[] tasks = executionGraph.getJobVertex(jobVertexId).getTaskVertices(); tasks[0].getCurrentExecutionAttempt().markFinished(); @@ -100,7 +107,7 @@ public void testAbortionIfPartiallyOperatorsFinishedVertexUsedUnionListState() Collections.singletonList( OperatorIDPair.generatedIDOnly(operatorId)), true) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex[] tasks = executionGraph.getJobVertex(jobVertexId).getTaskVertices(); CheckpointPlan checkpointPlan = createCheckpointPlan(executionGraph); @@ -156,7 +163,7 @@ public void testFulfillFinishedStates() throws Exception { OperatorIDPair.generatedIDOnly( partiallyFinishedOperatorId)), true) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); ExecutionVertex[] fullyFinishedVertexTasks = executionGraph.getJobVertex(fullyFinishedVertexId).getTaskVertices(); ExecutionVertex[] finishedOnRestoreVertexTasks = @@ -196,7 +203,7 @@ public void testFulfillFullyFinishedStatesWithCoordinator() throws Exception { Collections.singletonList( OperatorIDPair.generatedIDOnly(finishedOperatorID)), true) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); executionGraph .getJobVertex(finishedJobVertexID) .getTaskVertices()[0] diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultSchedulerCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultSchedulerCheckpointCoordinatorTest.java index 7676bf49e0f73..1fe9112284961 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultSchedulerCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultSchedulerCheckpointCoordinatorTest.java @@ -35,12 +35,16 @@ import org.apache.flink.runtime.scheduler.SchedulerBase; import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.hamcrest.Matchers; +import org.junit.ClassRule; import org.junit.Test; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; @@ -51,6 +55,10 @@ */ public class DefaultSchedulerCheckpointCoordinatorTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + /** Tests that the checkpoint coordinator is shut down if the execution graph is failed. */ @Test public void testClosingSchedulerShutsDownCheckpointCoordinatorOnFailedExecutionGraph() @@ -200,8 +208,10 @@ private DefaultScheduler createSchedulerAndEnableCheckpointing( .setJobCheckpointingSettings(checkpointingSettings) .build(); - return SchedulerTestingUtils.newSchedulerBuilder( - jobGraph, ComponentMainThreadExecutorServiceAdapter.forMainThread()) + return new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()) .setCheckpointRecoveryFactory(new TestingCheckpointRecoveryFactory(store, counter)) .setRpcTimeout(timeout) .build(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java index ded06ff187d04..45c980c193f5c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/FailoverStrategyCheckpointCoordinatorTest.java @@ -24,14 +24,18 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.Executors; import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import java.util.Collections; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadLocalRandom; import static org.junit.Assert.assertEquals; @@ -40,6 +44,11 @@ /** Tests for actions of {@link CheckpointCoordinator} on task failures. */ public class FailoverStrategyCheckpointCoordinatorTest extends TestLogger { + + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private ManuallyTriggeredScheduledExecutor manualThreadExecutor; @Before @@ -58,7 +67,7 @@ public void testAbortPendingCheckpointsWithTriggerValidation() throws Exception new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(new JobVertexID()) .setTransitToRunning(false) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); CheckpointCoordinatorConfiguration checkpointCoordinatorConfiguration = new CheckpointCoordinatorConfiguration( Integer.MAX_VALUE, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperationTest.java index 629150cecc89b..7aa742e211cd6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/StateAssignmentOperationTest.java @@ -43,9 +43,12 @@ import org.apache.flink.runtime.state.OperatorStreamStateHandle; import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import java.util.Arrays; @@ -57,6 +60,7 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -86,6 +90,9 @@ /** Tests to verify state assignment operation. */ public class StateAssignmentOperationTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); private static final int MAX_P = 256; @@ -926,7 +933,9 @@ private Map toExecutionVertices(JobVertex... job throws JobException, JobExecutionException { JobGraph jobGraph = JobGraphTestUtils.streamingJobGraph(jobVertices); ExecutionGraph eg = - TestingDefaultExecutionGraphBuilder.newBuilder().setJobGraph(jobGraph).build(); + TestingDefaultExecutionGraphBuilder.newBuilder() + .setJobGraph(jobGraph) + .build(EXECUTOR_RESOURCE.getExecutor()); return Arrays.stream(jobVertices) .collect( Collectors.toMap( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/VertexFinishedStateCheckerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/VertexFinishedStateCheckerTest.java index 379280e79e505..c540cd9f17468 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/VertexFinishedStateCheckerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/VertexFinishedStateCheckerTest.java @@ -27,8 +27,11 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.FlinkRuntimeException; +import org.junit.ClassRule; import org.junit.Test; import java.util.Arrays; @@ -36,6 +39,7 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; import static java.util.Collections.singletonList; import static org.hamcrest.CoreMatchers.equalTo; @@ -46,6 +50,10 @@ /** This tests verifies the checking logic of {@link VertexFinishedStateChecker}. */ public class VertexFinishedStateCheckerTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Test public void testRestoringPartiallyFinishedChainsFailsWithoutUidHash() throws Exception { // If useUidHash is set to false, the operator states would still be keyed with the @@ -72,7 +80,7 @@ private void testRestoringPartiallyFinishedChainsFails(boolean useUidHash) throw new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() .addJobVertex(jobVertexID2, 1, 1, singletonList(op3), true) .addJobVertex(jobVertexID1, 1, 1, Arrays.asList(op1, op2), true) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); Map operatorStates = new HashMap<>(); operatorStates.put( @@ -261,7 +269,7 @@ private void testAddingOperatorsBeforePartiallyOrFullyFinishedOne( .addJobVertex(vertex1, true) .addJobVertex(vertex2, false) .setDistributionPattern(distributionPatterns[0]) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); // Adds the additional edges for (int i = 1; i < distributionPatterns.length; ++i) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactoryTest.java index 39b927963d505..edf1261b494e2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorFactoryTest.java @@ -40,9 +40,12 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.CompressedSerializedValue; import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; import org.junit.Test; import java.io.IOException; @@ -50,6 +53,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; @@ -57,6 +61,9 @@ /** Tests for {@link TaskDeploymentDescriptorFactory}. */ public class TaskDeploymentDescriptorFactoryTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); private static final int PARALLELISM = 4; @@ -150,7 +157,7 @@ private static ExecutionGraph createExecutionGraph( return TestingDefaultExecutionGraphBuilder.newBuilder() .setJobGraph(jobGraph) .setBlobWriter(blobWriter) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); } private static TaskDeploymentDescriptor createTaskDeploymentDescriptor(ExecutionVertex ev) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ExecutionGraphInfoStoreTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ExecutionGraphInfoStoreTestUtils.java index 048fd871ae69f..d01469b9af437 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ExecutionGraphInfoStoreTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/ExecutionGraphInfoStoreTestUtils.java @@ -42,8 +42,8 @@ import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; import org.apache.flink.runtime.security.token.DelegationTokenManager; import org.apache.flink.runtime.webmonitor.retriever.MetricQueryServiceRetriever; -import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.concurrent.ScheduledExecutor; import org.apache.flink.shaded.guava30.com.google.common.base.Ticker; @@ -175,15 +175,21 @@ public void invoke() throws Exception { /** MiniCluster with specified {@link ExecutionGraphInfoStore}. */ static class PersistingMiniCluster extends MiniCluster { @Nullable private final File rootDir; + private final ScheduledExecutor scheduledExecutor; - PersistingMiniCluster(MiniClusterConfiguration miniClusterConfiguration) { - this(miniClusterConfiguration, null); + PersistingMiniCluster( + MiniClusterConfiguration miniClusterConfiguration, + ScheduledExecutor scheduledExecutor) { + this(miniClusterConfiguration, null, scheduledExecutor); } PersistingMiniCluster( - MiniClusterConfiguration miniClusterConfiguration, @Nullable File rootDir) { + MiniClusterConfiguration miniClusterConfiguration, + @Nullable File rootDir, + ScheduledExecutor scheduledExecutor) { super(miniClusterConfiguration); this.rootDir = rootDir; + this.scheduledExecutor = scheduledExecutor; } @Override @@ -210,7 +216,8 @@ static class PersistingMiniCluster extends MiniCluster { switch (jobStoreType) { case File: { - executionGraphInfoStore = createDefaultExecutionGraphInfoStore(rootDir); + executionGraphInfoStore = + createDefaultExecutionGraphInfoStore(rootDir, scheduledExecutor); break; } case Memory: @@ -242,14 +249,14 @@ static class PersistingMiniCluster extends MiniCluster { } } - static FileExecutionGraphInfoStore createDefaultExecutionGraphInfoStore(File storageDirectory) - throws IOException { + static FileExecutionGraphInfoStore createDefaultExecutionGraphInfoStore( + File storageDirectory, ScheduledExecutor scheduledExecutor) throws IOException { return new FileExecutionGraphInfoStore( storageDirectory, Time.hours(1L), Integer.MAX_VALUE, 10000L, - TestingUtils.defaultScheduledExecutor(), + scheduledExecutor, Ticker.systemTicker()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FileExecutionGraphInfoStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FileExecutionGraphInfoStoreTest.java index 38d4e46ee9d70..be079eec74911 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FileExecutionGraphInfoStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/FileExecutionGraphInfoStoreTest.java @@ -33,8 +33,10 @@ import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; import org.apache.flink.runtime.util.ManualTicker; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; +import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; import org.apache.flink.shaded.guava30.com.google.common.base.Ticker; import org.apache.flink.shaded.guava30.com.google.common.cache.LoadingCache; @@ -50,6 +52,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -62,6 +65,11 @@ /** Tests for the {@link FileExecutionGraphInfoStore}. */ public class FileExecutionGraphInfoStoreTest extends TestLogger { + + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); /** @@ -85,7 +93,9 @@ public void testUnknownGet() throws IOException { final File rootDir = temporaryFolder.newFolder(); try (final FileExecutionGraphInfoStore executionGraphStore = - createDefaultExecutionGraphInfoStore(rootDir)) { + createDefaultExecutionGraphInfoStore( + rootDir, + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()))) { assertThat(executionGraphStore.get(new JobID()), Matchers.nullValue()); } } @@ -108,7 +118,9 @@ public void testStoredJobsOverview() throws IOException { final File rootDir = temporaryFolder.newFolder(); try (final FileExecutionGraphInfoStore executionGraphInfoStore = - createDefaultExecutionGraphInfoStore(rootDir)) { + createDefaultExecutionGraphInfoStore( + rootDir, + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()))) { for (ExecutionGraphInfo executionGraphInfo : executionGraphInfos) { executionGraphInfoStore.put(executionGraphInfo); } @@ -131,7 +143,9 @@ public void testAvailableJobDetails() throws IOException { final File rootDir = temporaryFolder.newFolder(); try (final FileExecutionGraphInfoStore executionGraphInfoStore = - createDefaultExecutionGraphInfoStore(rootDir)) { + createDefaultExecutionGraphInfoStore( + rootDir, + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()))) { for (ExecutionGraphInfo executionGraphInfo : executionGraphInfos) { executionGraphInfoStore.put(executionGraphInfo); } @@ -200,7 +214,9 @@ public void testCloseCleansUp() throws IOException { assertThat(rootDir.listFiles().length, Matchers.equalTo(0)); try (final FileExecutionGraphInfoStore executionGraphInfoStore = - createDefaultExecutionGraphInfoStore(rootDir)) { + createDefaultExecutionGraphInfoStore( + rootDir, + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()))) { assertThat(rootDir.listFiles().length, Matchers.equalTo(1)); @@ -231,7 +247,7 @@ public void testCacheLoading() throws IOException { Time.hours(1L), Integer.MAX_VALUE, 100L << 10, - TestingUtils.defaultScheduledExecutor(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()), Ticker.systemTicker())) { final LoadingCache executionGraphInfoCache = @@ -296,7 +312,7 @@ public void testMaximumCapacity() throws IOException { Time.hours(1L), maxCapacity, 10000L, - TestingUtils.defaultScheduledExecutor(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()), Ticker.systemTicker())) { for (ExecutionGraphInfo executionGraphInfo : oldExecutionGraphInfos) { @@ -324,7 +340,9 @@ public void testPutSuspendedJobOnClusterShutdown() throws Exception { File rootDir = temporaryFolder.newFolder(); try (final MiniCluster miniCluster = new ExecutionGraphInfoStoreTestUtils.PersistingMiniCluster( - new MiniClusterConfiguration.Builder().build(), rootDir)) { + new MiniClusterConfiguration.Builder().build(), + rootDir, + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()))) { miniCluster.start(); final JobVertex vertex = new JobVertex("blockingVertex"); // The adaptive scheduler expects that every vertex has a configured parallelism @@ -344,7 +362,9 @@ private void assertPutJobGraphWithStatus(JobStatus jobStatus) throws IOException final File rootDir = temporaryFolder.newFolder(); try (final FileExecutionGraphInfoStore executionGraphStore = - createDefaultExecutionGraphInfoStore(rootDir)) { + createDefaultExecutionGraphInfoStore( + rootDir, + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()))) { final File storageDirectory = executionGraphStore.getStorageDir(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobDispatcherITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobDispatcherITCase.java index f6fbcdde6bc1a..908a0eca3352c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobDispatcherITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobDispatcherITCase.java @@ -46,12 +46,14 @@ import org.apache.flink.runtime.scheduler.adaptive.AdaptiveSchedulerClusterITCase; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorExtension; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.io.TempDir; import java.io.IOException; @@ -62,6 +64,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; import java.util.function.Supplier; import static java.nio.file.StandardOpenOption.CREATE; @@ -70,6 +73,9 @@ /** An integration test which recovers from checkpoint after regaining the leadership. */ @ExtendWith(TestLoggerExtension.class) public class JobDispatcherITCase { + @RegisterExtension + static final TestExecutorExtension EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorExtension(); private Supplier createJobModeDispatcherResourceManagerComponentFactorySupplier( @@ -98,7 +104,7 @@ public void testRecoverFromCheckpointAfterLosingAndRegainingLeadership(@TempDir .setConfiguration(configuration) .build(); final EmbeddedHaServicesWithLeadershipControl haServices = - new EmbeddedHaServicesWithLeadershipControl(TestingUtils.defaultExecutor()); + new EmbeddedHaServicesWithLeadershipControl(EXECUTOR_RESOURCE.getExecutor()); final Configuration newConfiguration = new Configuration(clusterConfiguration.getConfiguration()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MemoryExecutionGraphInfoStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MemoryExecutionGraphInfoStoreTest.java index b7f3187d132e8..c1cb28833c035 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MemoryExecutionGraphInfoStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MemoryExecutionGraphInfoStoreTest.java @@ -35,17 +35,21 @@ import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; import org.apache.flink.runtime.util.ManualTicker; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; +import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; import org.apache.flink.shaded.guava30.com.google.common.base.Ticker; import org.hamcrest.Matchers; +import org.junit.ClassRule; import org.junit.Test; import java.io.IOException; import java.util.Collection; import java.util.List; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -58,6 +62,10 @@ /** Tests for the {@link MemoryExecutionGraphInfoStore}. */ public class MemoryExecutionGraphInfoStoreTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + /** * Tests that we can put {@link ExecutionGraphInfo} into the {@link * MemoryExecutionGraphInfoStore} and that the graph is persisted. @@ -241,7 +249,8 @@ public void testPutSuspendedJobOnClusterShutdown() throws Exception { new ExecutionGraphInfoStoreTestUtils.PersistingMiniCluster( new MiniClusterConfiguration.Builder() .setConfiguration(configuration) - .build())) { + .build(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()))) { miniCluster.start(); final JobVertex vertex = new JobVertex("blockingVertex"); // The adaptive scheduler expects that every vertex has a configured parallelism @@ -285,7 +294,7 @@ private MemoryExecutionGraphInfoStore createMemoryExecutionGraphInfoStore( return new MemoryExecutionGraphInfoStore( expirationTime, maximumCapacity, - TestingUtils.defaultScheduledExecutor(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()), Ticker.systemTicker()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java index ff1b2b7f52a6e..2c25c7e22bf73 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerITCase.java @@ -53,6 +53,7 @@ import org.apache.flink.runtime.util.LeaderConnectionInfo; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; @@ -67,6 +68,7 @@ import java.util.Collection; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import static org.hamcrest.Matchers.contains; @@ -87,6 +89,10 @@ public class DefaultDispatcherRunnerITCase extends TestLogger { @ClassRule public static BlobServerResource blobServerResource = new BlobServerResource(); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private JobGraph jobGraph; private TestingLeaderElectionService dispatcherLeaderElectionService; @@ -266,7 +272,7 @@ private DispatcherRunner createDispatcherRunner() throws Exception { dispatcherLeaderElectionService, fatalErrorHandler, new TestingJobPersistenceComponentFactory(jobGraphStore, jobResultStore), - TestingUtils.defaultExecutor(), + EXECUTOR_RESOURCE.getExecutor(), rpcServiceResource.getTestingRpcService(), partialDispatcherServices); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java index dde393d4fa782..4a3dca8f97448 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java @@ -55,6 +55,7 @@ import org.apache.flink.runtime.util.ZooKeeperUtils; import org.apache.flink.runtime.zookeeper.ZooKeeperResource; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -73,6 +74,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.Matchers.emptyArray; import static org.hamcrest.Matchers.is; @@ -90,6 +92,10 @@ public class ZooKeeperDefaultDispatcherRunnerTest extends TestLogger { @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @ClassRule public static TestingRpcServiceResource testingRpcServiceResource = new TestingRpcServiceResource(); @@ -240,7 +246,7 @@ private DispatcherRunner createDispatcherRunner( dispatcherLeaderElectionService, fatalErrorHandler, jobPersistenceComponentFactory, - TestingUtils.defaultExecutor(), + EXECUTOR_RESOURCE.getExecutor(), rpcService, partialDispatcherServices); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AllVerticesIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AllVerticesIteratorTest.java index 974b3c11286e3..93556ee8c8f72 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AllVerticesIteratorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AllVerticesIteratorTest.java @@ -20,14 +20,22 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import java.util.Arrays; +import java.util.concurrent.ScheduledExecutorService; public class AllVerticesIteratorTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Test public void testAllVertices() { try { @@ -47,7 +55,9 @@ public void testAllVertices() { v3.setParallelism(3); v4.setParallelism(2); - ExecutionGraph eg = ExecutionGraphTestUtils.createSimpleTestGraph(v1, v2, v3, v4); + ExecutionGraph eg = + ExecutionGraphTestUtils.createExecutionGraph( + EXECUTOR_RESOURCE.getExecutor(), v1, v2, v3, v4); ExecutionJobVertex ejv1 = eg.getJobVertex(v1.getID()); ExecutionJobVertex ejv2 = eg.getJobVertex(v2.getID()); ExecutionJobVertex ejv3 = eg.getJobVertex(v3.getID()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java index a79592795e24e..6819ef941c752 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java @@ -41,11 +41,14 @@ import org.apache.flink.runtime.scheduler.SchedulerBase; import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import java.io.IOException; @@ -54,6 +57,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.concurrent.ScheduledExecutorService; import java.util.function.Function; import static java.util.Arrays.asList; @@ -68,6 +72,10 @@ /** Tests for the {@link ArchivedExecutionGraph}. */ public class ArchivedExecutionGraphTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static ExecutionGraph runtimeGraph; @BeforeClass @@ -118,7 +126,9 @@ public static void setupExecutionGraph() throws Exception { SchedulerBase scheduler = SchedulerTestingUtils.createScheduler( - jobGraph, ComponentMainThreadExecutorServiceAdapter.forMainThread()); + jobGraph, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()); runtimeGraph = scheduler.getExecutionGraph(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphConstructionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphConstructionTest.java index 243b921811c21..ee4383c82964d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphConstructionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphConstructionTest.java @@ -32,9 +32,12 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.scheduler.SchedulerBase; import org.apache.flink.runtime.scheduler.strategy.ConsumedPartitionGroup; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.shaded.guava30.com.google.common.collect.Sets; +import org.junit.ClassRule; import org.junit.Test; import org.mockito.Matchers; @@ -46,6 +49,7 @@ import java.util.List; import java.util.Objects; import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -62,17 +66,20 @@ * builds {@link DistributionPattern#ALL_TO_ALL} connections correctly. */ public class DefaultExecutionGraphConstructionTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); private ExecutionGraph createDefaultExecutionGraph(List vertices) throws Exception { return TestingDefaultExecutionGraphBuilder.newBuilder() .setVertexParallelismStore(SchedulerBase.computeVertexParallelismStore(vertices)) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); } private ExecutionGraph createDynamicExecutionGraph(List vertices) throws Exception { return TestingDefaultExecutionGraphBuilder.newBuilder() .setVertexParallelismStore(SchedulerBase.computeVertexParallelismStore(vertices)) - .buildDynamicGraph(); + .buildDynamicGraph(EXECUTOR_RESOURCE.getExecutor()); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java index a83578c8a63f1..381b8d4d023e3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java @@ -64,11 +64,14 @@ import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.DirectScheduledExecutorService; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.util.function.FunctionUtils; import org.hamcrest.Description; import org.hamcrest.TypeSafeMatcher; +import org.junit.ClassRule; import org.junit.Test; import java.util.ArrayList; @@ -81,6 +84,7 @@ import java.util.Map; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import static junit.framework.TestCase.assertTrue; import static org.junit.Assert.assertEquals; @@ -92,6 +96,10 @@ /** Tests for {@link DefaultExecutionGraph} deployment. */ public class DefaultExecutionGraphDeploymentTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + /** BLOB server instance to use for the job graph. */ protected BlobWriter blobWriter = VoidBlobWriter.getInstance(); @@ -159,10 +167,8 @@ public void testBuildDeploymentDescriptor() throws Exception { DefaultExecutionGraph eg = TestingDefaultExecutionGraphBuilder.newBuilder() .setJobGraph(jobGraph) - .setFutureExecutor(executor) - .setIoExecutor(executor) .setBlobWriter(blobWriter) - .build(); + .build(executor); eg.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); @@ -471,8 +477,10 @@ public void testNoResourceAvailableFailure() throws Exception { // execution graph that executes actions synchronously final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( - graph, ComponentMainThreadExecutorServiceAdapter.forMainThread()) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + graph, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( TestingPhysicalSlotProvider @@ -530,9 +538,10 @@ private SchedulerBase setupScheduler(JobVertex v1, int dop1, JobVertex v2, int d // execution graph that executes actions synchronously final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( + new SchedulerTestingUtils.DefaultSchedulerBuilder( JobGraphTestUtils.streamingJobGraph(v1, v2), - ComponentMainThreadExecutorServiceAdapter.forMainThread()) + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory()) .setFutureExecutor(executorService) @@ -615,8 +624,10 @@ public void testExecutionGraphIsDeployedInTopologicalOrder() throws Exception { final TestingPhysicalSlotProvider physicalSlotProvider = TestingPhysicalSlotProvider.createWithoutImmediatePhysicalSlotCreation(); final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( - jobGraph, ComponentMainThreadExecutorServiceAdapter.forMainThread()) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( physicalSlotProvider)) @@ -681,7 +692,7 @@ private ExecutionGraph createExecutionGraph(Configuration configuration) throws .setJobGraph(jobGraph) .setJobMasterConfig(configuration) .setBlobWriter(blobWriter) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); } private static final class ExecutionStageMatcher diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentWithSmallBlobCacheSizeLimitTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentWithSmallBlobCacheSizeLimitTest.java index d2a4bc568567c..e2aa7a5809d6d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentWithSmallBlobCacheSizeLimitTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentWithSmallBlobCacheSizeLimitTest.java @@ -176,10 +176,8 @@ private ExecutionGraph createAndSetupExecutionGraph(int numberOfVertices, int pa final DefaultExecutionGraph eg = TestingDefaultExecutionGraphBuilder.newBuilder() .setJobGraph(jobGraph) - .setFutureExecutor(executor) - .setIoExecutor(executor) .setBlobWriter(blobWriter) - .build(); + .build(executor); eg.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphRescalingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphRescalingTest.java index de1405471f3cb..30ba54059f9ee 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphRescalingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphRescalingTest.java @@ -26,12 +26,16 @@ import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; import org.junit.Test; import java.util.Arrays; import java.util.Collections; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertThat; @@ -42,6 +46,9 @@ * ExecutionGraph}s are correct. */ public class DefaultExecutionGraphRescalingTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); @Test public void testExecutionGraphArbitraryDopConstructionTest() throws Exception { @@ -53,7 +60,9 @@ public void testExecutionGraphArbitraryDopConstructionTest() throws Exception { final JobGraph jobGraph = JobGraphTestUtils.streamingJobGraph(jobVertices); ExecutionGraph eg = - TestingDefaultExecutionGraphBuilder.newBuilder().setJobGraph(jobGraph).build(); + TestingDefaultExecutionGraphBuilder.newBuilder() + .setJobGraph(jobGraph) + .build(EXECUTOR_RESOURCE.getExecutor()); for (JobVertex jv : jobVertices) { assertThat(jv.getParallelism(), is(initialParallelism)); @@ -68,7 +77,10 @@ public void testExecutionGraphArbitraryDopConstructionTest() throws Exception { jv.setParallelism(scaleDownParallelism); } - eg = TestingDefaultExecutionGraphBuilder.newBuilder().setJobGraph(jobGraph).build(); + eg = + TestingDefaultExecutionGraphBuilder.newBuilder() + .setJobGraph(jobGraph) + .build(EXECUTOR_RESOURCE.getExecutor()); for (JobVertex jv : jobVertices) { assertThat(jv.getParallelism(), is(1)); @@ -83,7 +95,10 @@ public void testExecutionGraphArbitraryDopConstructionTest() throws Exception { jv.setParallelism(scaleUpParallelism); } - eg = TestingDefaultExecutionGraphBuilder.newBuilder().setJobGraph(jobGraph).build(); + eg = + TestingDefaultExecutionGraphBuilder.newBuilder() + .setJobGraph(jobGraph) + .build(EXECUTOR_RESOURCE.getExecutor()); for (JobVertex jv : jobVertices) { assertThat(jv.getParallelism(), is(scaleUpParallelism)); @@ -113,7 +128,9 @@ public void testExecutionGraphConstructionFailsRescaleDopExceedMaxParallelism() try { // this should fail since we set the parallelism to maxParallelism + 1 - TestingDefaultExecutionGraphBuilder.newBuilder().setJobGraph(jobGraph).build(); + TestingDefaultExecutionGraphBuilder.newBuilder() + .setJobGraph(jobGraph) + .build(EXECUTOR_RESOURCE.getExecutor()); fail( "Building the ExecutionGraph with a parallelism higher than the max parallelism should fail."); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/EdgeManagerBuildUtilTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/EdgeManagerBuildUtilTest.java index 8c3c1ce9516cb..2cd5a8d1ce30d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/EdgeManagerBuildUtilTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/EdgeManagerBuildUtilTest.java @@ -24,13 +24,17 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.scheduler.SchedulerBase; import org.apache.flink.runtime.scheduler.strategy.ConsumerVertexGroup; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.commons.lang3.tuple.Pair; +import org.junit.ClassRule; import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.concurrent.ScheduledExecutorService; import static org.apache.flink.runtime.jobgraph.DistributionPattern.ALL_TO_ALL; import static org.apache.flink.runtime.jobgraph.DistributionPattern.POINTWISE; @@ -42,6 +46,9 @@ * DistributionPattern#ALL_TO_ALL}. */ public class EdgeManagerBuildUtilTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); @Test public void testGetMaxNumEdgesToTargetInPointwiseConnection() throws Exception { @@ -120,7 +127,7 @@ private Pair setupExecutionGraph( TestingDefaultExecutionGraphBuilder.newBuilder() .setVertexParallelismStore( SchedulerBase.computeVertexParallelismStore(ordered)) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); eg.attachJobGraph(ordered); return Pair.of(eg.getAllVertices().get(v1.getID()), eg.getAllVertices().get(v2.getID())); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/EdgeManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/EdgeManagerTest.java index 39582b3716f89..323a753a51150 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/EdgeManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/EdgeManagerTest.java @@ -28,16 +28,24 @@ import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; import org.apache.flink.runtime.scheduler.strategy.ConsumedPartitionGroup; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; +import org.junit.ClassRule; import org.junit.Test; import java.util.Objects; +import java.util.concurrent.ScheduledExecutorService; import static org.junit.Assert.assertEquals; /** Tests for {@link EdgeManager}. */ public class EdgeManagerTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Test public void testGetConsumedPartitionGroup() throws Exception { JobVertex v1 = new JobVertex("source"); @@ -55,7 +63,9 @@ public void testGetConsumedPartitionGroup() throws Exception { JobGraph jobGraph = JobGraphTestUtils.batchJobGraph(v1, v2); SchedulerBase scheduler = SchedulerTestingUtils.createScheduler( - jobGraph, ComponentMainThreadExecutorServiceAdapter.forMainThread()); + jobGraph, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()); ExecutionGraph eg = scheduler.getExecutionGraph(); ConsumedPartitionGroup groupRetrievedByDownstreamVertex = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphCoLocationRestartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphCoLocationRestartTest.java index 1d280cae408c1..d7a9d89557b25 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphCoLocationRestartTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphCoLocationRestartTest.java @@ -34,11 +34,15 @@ import org.apache.flink.runtime.scheduler.TestingPhysicalSlot; import org.apache.flink.runtime.scheduler.TestingPhysicalSlotProvider; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.FlinkException; +import org.junit.ClassRule; import org.junit.Test; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import java.util.function.Predicate; import static org.apache.flink.api.common.JobStatus.FINISHED; @@ -49,6 +53,10 @@ /** Tests that co-location constraints work as expected in the case of task restarts. */ public class ExecutionGraphCoLocationRestartTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static final int NUM_TASKS = 31; @Test @@ -72,8 +80,10 @@ public void testConstraintsAfterRestart() throws Exception { final ManuallyTriggeredScheduledExecutorService delayExecutor = new ManuallyTriggeredScheduledExecutorService(); final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( - jobGraph, ComponentMainThreadExecutorServiceAdapter.forMainThread()) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( TestingPhysicalSlotProvider.create( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphFinishTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphFinishTest.java index 363aaff5cc7e1..d23b13cc59f4f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphFinishTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphFinishTest.java @@ -25,19 +25,27 @@ import org.apache.flink.runtime.scheduler.SchedulerBase; import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; import org.junit.Test; import java.util.Arrays; import java.util.Iterator; import java.util.List; +import java.util.concurrent.ScheduledExecutorService; import static org.junit.Assert.assertEquals; /** Tests the finish behaviour of the {@link ExecutionGraph}. */ public class ExecutionGraphFinishTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Test public void testJobFinishes() throws Exception { @@ -47,8 +55,10 @@ public void testJobFinishes() throws Exception { ExecutionGraphTestUtils.createJobVertex("Task2", 2, NoOpInvokable.class)); SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( - jobGraph, ComponentMainThreadExecutorServiceAdapter.forMainThread()) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()) .build(); ExecutionGraph eg = scheduler.getExecutionGraph(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphPartitionReleaseTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphPartitionReleaseTest.java index c8543746b8765..79e5495380f07 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphPartitionReleaseTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphPartitionReleaseTest.java @@ -32,8 +32,11 @@ import org.apache.flink.runtime.scheduler.SchedulerBase; import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; import org.junit.Test; import java.util.ArrayDeque; @@ -52,6 +55,10 @@ */ public class ExecutionGraphPartitionReleaseTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static final ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); private static final TestingComponentMainThreadExecutor mainThreadExecutor = @@ -251,8 +258,10 @@ private SchedulerBase createScheduler( final JobGraph jobGraph = JobGraphTestUtils.batchJobGraph(vertices); final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( - jobGraph, mainThreadExecutor.getMainThreadExecutor()) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, + mainThreadExecutor.getMainThreadExecutor(), + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory()) .setPartitionTracker(partitionTracker) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java index 8caffe12f048f..facd4be2fbe2e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRestartTest.java @@ -44,15 +44,19 @@ import org.apache.flink.runtime.scheduler.SchedulerBase; import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import java.io.IOException; import java.util.Collections; import java.util.Iterator; +import java.util.concurrent.ScheduledExecutorService; import java.util.function.Consumer; import static org.hamcrest.Matchers.is; @@ -65,6 +69,10 @@ public class ExecutionGraphRestartTest extends TestLogger { private static final int NUM_TASKS = 31; + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static final ComponentMainThreadExecutor mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread(); @@ -99,7 +107,8 @@ public void testCancelAllPendingRequestWhileCanceling() throws Exception { "Task", NUM_TASKS + numTasksExceedSlotPool, NoOpInvokable.class); JobGraph graph = JobGraphTestUtils.streamingJobGraph(sender); SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder(graph, mainThreadExecutor) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + graph, mainThreadExecutor, EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( createExecutionSlotAllocatorFactory(slotPool)) .build(); @@ -127,7 +136,8 @@ public void testCancelAllPendingRequestWhileFailing() throws Exception { "Task", NUM_TASKS + numTasksExceedSlotPool, NoOpInvokable.class); JobGraph graph = JobGraphTestUtils.streamingJobGraph(sender); SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder(graph, mainThreadExecutor) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + graph, mainThreadExecutor, EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( createExecutionSlotAllocatorFactory(slotPool)) .build(); @@ -149,7 +159,10 @@ public void testCancelWhileRestarting() throws Exception { // We want to manually control the restart and delay try (SlotPool slotPool = SlotPoolUtils.createDeclarativeSlotPoolBridge()) { SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder(createJobGraph(), mainThreadExecutor) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + createJobGraph(), + mainThreadExecutor, + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( createExecutionSlotAllocatorFactory(slotPool)) .setRestartBackoffTimeStrategy( @@ -189,7 +202,10 @@ private static ResourceID offerSlots(SlotPool slotPool, int numSlots) { public void testCancelWhileFailing() throws Exception { try (SlotPool slotPool = SlotPoolUtils.createDeclarativeSlotPoolBridge()) { SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder(createJobGraph(), mainThreadExecutor) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + createJobGraph(), + mainThreadExecutor, + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( createExecutionSlotAllocatorFactory(slotPool)) .setRestartBackoffTimeStrategy( @@ -224,7 +240,10 @@ public void testCancelWhileFailing() throws Exception { public void testFailWhileCanceling() throws Exception { try (SlotPool slotPool = SlotPoolUtils.createDeclarativeSlotPoolBridge()) { SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder(createJobGraph(), mainThreadExecutor) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + createJobGraph(), + mainThreadExecutor, + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( createExecutionSlotAllocatorFactory(slotPool)) .setRestartBackoffTimeStrategy( @@ -272,7 +291,8 @@ public void testFailingExecutionAfterRestart() throws Exception { try (SlotPool slotPool = SlotPoolUtils.createDeclarativeSlotPoolBridge()) { SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder(jobGraph, mainThreadExecutor) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, mainThreadExecutor, EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( createExecutionSlotAllocatorFactory(slotPool)) .setRestartBackoffTimeStrategy( @@ -331,8 +351,10 @@ public void testFailingExecutionAfterRestart() throws Exception { public void testFailExecutionAfterCancel() throws Exception { try (SlotPool slotPool = SlotPoolUtils.createDeclarativeSlotPoolBridge()) { SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( - createJobGraphToCancel(), mainThreadExecutor) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + createJobGraphToCancel(), + mainThreadExecutor, + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( createExecutionSlotAllocatorFactory(slotPool)) .setRestartBackoffTimeStrategy( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java index 28e96c6627abc..5804f782ff9e4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphSuspendTest.java @@ -29,11 +29,16 @@ import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; import org.apache.flink.runtime.scheduler.TestingPhysicalSlotProvider; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; +import org.junit.ClassRule; import org.junit.Test; +import java.util.concurrent.ScheduledExecutorService; + import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -41,6 +46,10 @@ /** Validates that suspending out of various states works correctly. */ public class ExecutionGraphSuspendTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + /** * Going into SUSPENDED out of CREATED should immediately cancel everything and not send out RPC * calls. @@ -228,9 +237,10 @@ public void testSuspendWhileRestarting() throws Exception { final ManuallyTriggeredScheduledExecutor taskRestartExecutor = new ManuallyTriggeredScheduledExecutor(); final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( + new SchedulerTestingUtils.DefaultSchedulerBuilder( JobGraphTestUtils.emptyJobGraph(), - ComponentMainThreadExecutorServiceAdapter.forMainThread()) + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()) .setRestartBackoffTimeStrategy( new TestRestartBackoffTimeStrategy(true, Long.MAX_VALUE)) .setDelayExecutor(taskRestartExecutor) @@ -308,9 +318,10 @@ private static SchedulerBase createScheduler(TaskManagerGateway gateway, int par vertex.setParallelism(parallelism); final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( + new SchedulerTestingUtils.DefaultSchedulerBuilder( JobGraphTestUtils.streamingJobGraph(vertex), - ComponentMainThreadExecutorServiceAdapter.forMainThread()) + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( TestingPhysicalSlotProvider diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java index d3fb2227b7beb..df4664bf6ac22 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java @@ -37,7 +37,6 @@ import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.DirectScheduledExecutorService; -import org.apache.flink.testutils.TestingUtils; import javax.annotation.Nullable; @@ -305,19 +304,6 @@ public static void setVertexResource(ExecutionVertex vertex, LogicalSlot slot) { // Mocking ExecutionGraph // ------------------------------------------------------------------------ - /** Creates an execution graph with on job vertex of parallelism 10. */ - public static ExecutionGraph createSimpleTestGraph() throws Exception { - JobVertex vertex = createNoOpVertex(10); - - return createSimpleTestGraph(vertex); - } - - /** Creates an execution graph containing the given vertices. */ - public static DefaultExecutionGraph createSimpleTestGraph(JobVertex... vertices) - throws Exception { - return createExecutionGraph(TestingUtils.defaultExecutor(), vertices); - } - public static DefaultExecutionGraph createExecutionGraph( ScheduledExecutorService executor, JobVertex... vertices) throws Exception { @@ -334,10 +320,8 @@ public static DefaultExecutionGraph createExecutionGraph( DefaultExecutionGraph executionGraph = TestingDefaultExecutionGraphBuilder.newBuilder() .setJobGraph(JobGraphTestUtils.streamingJobGraph(vertices)) - .setFutureExecutor(executor) - .setIoExecutor(executor) .setRpcTimeout(timeout) - .build(); + .build(executor); executionGraph.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); return executionGraph; } @@ -407,10 +391,10 @@ public static ExecutionJobVertex getExecutionJobVertex( JobGraph jobGraph = JobGraphTestUtils.batchJobGraph(jobVertex); SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( - jobGraph, ComponentMainThreadExecutorServiceAdapter.forMainThread()) - .setIoExecutor(executor) - .setFutureExecutor(executor) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + executor) .build(); return scheduler.getExecutionJobVertex(jobVertex.getID()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphVariousFailuesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphVariousFailuesTest.java index 9f84e0d07e3f7..eaf80d67bb67f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphVariousFailuesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphVariousFailuesTest.java @@ -25,10 +25,15 @@ import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.scheduler.SchedulerBase; import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; import org.junit.Test; +import java.util.concurrent.ScheduledExecutorService; + import static org.hamcrest.Matchers.containsString; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -36,6 +41,10 @@ public class ExecutionGraphVariousFailuesTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + /** * Tests that a failing notifyPartitionDataAvailable call with a non-existing execution attempt * id, will not fail the execution graph. @@ -43,9 +52,10 @@ public class ExecutionGraphVariousFailuesTest extends TestLogger { @Test public void testFailingNotifyPartitionDataAvailable() throws Exception { final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( + new SchedulerTestingUtils.DefaultSchedulerBuilder( JobGraphTestUtils.emptyJobGraph(), - ComponentMainThreadExecutorServiceAdapter.forMainThread()) + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()) .build(); scheduler.startScheduling(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertexTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertexTest.java index cbd58b250e9b6..8f2a35cece514 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertexTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertexTest.java @@ -28,11 +28,15 @@ import org.apache.flink.runtime.scheduler.VertexParallelismInformation; import org.apache.flink.runtime.scheduler.VertexParallelismStore; import org.apache.flink.runtime.scheduler.adaptivebatch.AdaptiveBatchScheduler; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import java.util.Collections; +import java.util.concurrent.ScheduledExecutorService; import static org.apache.flink.core.testutils.CommonTestUtils.assertThrows; import static org.hamcrest.CoreMatchers.is; @@ -40,6 +44,10 @@ /** Test for {@link ExecutionJobVertex} */ public class ExecutionJobVertexTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Test public void testParallelismGreaterThanMaxParallelism() { JobVertex jobVertex = new JobVertex("testVertex"); @@ -209,7 +217,9 @@ public static ExecutionJobVertex createDynamicExecutionJobVertex( jobVertex.setParallelism(parallelism); } - final DefaultExecutionGraph eg = TestingDefaultExecutionGraphBuilder.newBuilder().build(); + final DefaultExecutionGraph eg = + TestingDefaultExecutionGraphBuilder.newBuilder() + .build(EXECUTOR_RESOURCE.getExecutor()); final VertexParallelismStore vertexParallelismStore = AdaptiveBatchScheduler.computeVertexParallelismStoreForDynamicGraph( Collections.singletonList(jobVertex), defaultMaxParallelism); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java index 3a01a4097b5a3..62f907f78ccd7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionPartitionLifecycleTest.java @@ -48,6 +48,8 @@ import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.junit.ClassRule; @@ -61,6 +63,7 @@ import java.util.Queue; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import java.util.function.Consumer; import static org.hamcrest.Matchers.contains; @@ -74,8 +77,8 @@ public class ExecutionPartitionLifecycleTest extends TestLogger { @ClassRule - public static final TestingComponentMainThreadExecutor.Resource EXECUTOR_RESOURCE = - new TestingComponentMainThreadExecutor.Resource(); + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); private Execution execution; private ResultPartitionDeploymentDescriptor descriptor; @@ -276,8 +279,10 @@ private void setupExecutionGraphAndStartRunningJob( final JobGraph jobGraph = JobGraphTestUtils.batchJobGraph(producerVertex, consumerVertex); final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( - jobGraph, ComponentMainThreadExecutorServiceAdapter.forMainThread()) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( physicalSlotProvider)) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java index d3de904bbf6ee..9f11c7276325f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionTest.java @@ -31,6 +31,8 @@ import org.apache.flink.runtime.scheduler.TestingPhysicalSlot; import org.apache.flink.runtime.scheduler.TestingPhysicalSlotProvider; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; @@ -40,6 +42,7 @@ import javax.annotation.Nonnull; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; @@ -51,11 +54,15 @@ public class ExecutionTest extends TestLogger { @ClassRule - public static final TestingComponentMainThreadExecutor.Resource EXECUTOR_RESOURCE = + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + + @ClassRule + public static final TestingComponentMainThreadExecutor.Resource MAIN_EXECUTOR_RESOURCE = new TestingComponentMainThreadExecutor.Resource(); private final TestingComponentMainThreadExecutor testMainThreadUtil = - EXECUTOR_RESOURCE.getComponentMainThreadTestExecutor(); + MAIN_EXECUTOR_RESOURCE.getComponentMainThreadTestExecutor(); /** * Checks that the {@link Execution} termination future is only completed after the assigned @@ -72,9 +79,10 @@ public void testTerminationFutureIsCompletedAfterSlotRelease() throws Exception final TestingPhysicalSlotProvider physicalSlotProvider = TestingPhysicalSlotProvider.createWithLimitedAmountOfPhysicalSlots(1); final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( + new SchedulerTestingUtils.DefaultSchedulerBuilder( JobGraphTestUtils.streamingJobGraph(jobVertex), - ComponentMainThreadExecutorServiceAdapter.forMainThread()) + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( physicalSlotProvider)) @@ -115,9 +123,10 @@ public void testTaskRestoreStateIsNulledAfterDeployment() throws Exception { final JobVertexID jobVertexId = jobVertex.getID(); final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( + new SchedulerTestingUtils.DefaultSchedulerBuilder( JobGraphTestUtils.streamingJobGraph(jobVertex), - ComponentMainThreadExecutorServiceAdapter.forMainThread()) + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( TestingPhysicalSlotProvider @@ -159,9 +168,10 @@ public void testCanceledExecutionReturnsSlot() throws Exception { .withTaskManagerGateway(taskManagerGateway) .build())); final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( + new SchedulerTestingUtils.DefaultSchedulerBuilder( JobGraphTestUtils.streamingJobGraph(jobVertex), - testMainThreadUtil.getMainThreadExecutor()) + testMainThreadUtil.getMainThreadExecutor(), + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( physicalSlotProvider)) @@ -198,9 +208,10 @@ public void testSlotReleaseAtomicallyReleasesExecution() throws Exception { final TestingPhysicalSlotProvider physicalSlotProvider = TestingPhysicalSlotProvider.createWithLimitedAmountOfPhysicalSlots(1); final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( + new SchedulerTestingUtils.DefaultSchedulerBuilder( JobGraphTestUtils.streamingJobGraph(jobVertex), - testMainThreadUtil.getMainThreadExecutor()) + testMainThreadUtil.getMainThreadExecutor(), + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( physicalSlotProvider)) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java index a8d8d0f1bcef6..fc632f5d63287 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java @@ -29,13 +29,17 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.scheduler.SchedulerBase; import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; +import org.junit.ClassRule; import org.junit.Test; import java.io.IOException; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createNoOpVertex; import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionVertex; @@ -49,6 +53,10 @@ /** Tests for cancelling {@link ExecutionVertex ExecutionVertices}. */ public class ExecutionVertexCancelTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + // -------------------------------------------------------------------------------------------- // Canceling in different states // -------------------------------------------------------------------------------------------- @@ -242,7 +250,8 @@ public void testSendCancelAndReceiveFail() throws Exception { final SchedulerBase scheduler = SchedulerTestingUtils.createScheduler( JobGraphTestUtils.streamingJobGraph(createNoOpVertex(10)), - ComponentMainThreadExecutorServiceAdapter.forMainThread()); + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()); final ExecutionGraph graph = scheduler.getExecutionGraph(); scheduler.startScheduling(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java index f39460d318fb8..8c18a448406b8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java @@ -33,12 +33,16 @@ import org.apache.flink.runtime.scheduler.TestingPhysicalSlot; import org.apache.flink.runtime.scheduler.TestingPhysicalSlotProvider; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; import org.junit.Test; import java.util.Collection; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertFalse; @@ -46,6 +50,10 @@ /** Tests for the {@link ExecutionVertex}. */ public class ExecutionVertexTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Test public void testResetForNewExecutionReleasesPartitions() throws Exception { final JobVertex producerJobVertex = ExecutionGraphTestUtils.createNoOpVertex(1); @@ -64,8 +72,10 @@ public void testResetForNewExecutionReleasesPartitions() throws Exception { final JobGraph jobGraph = JobGraphTestUtils.streamingJobGraph(producerJobVertex, consumerJobVertex); final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( - jobGraph, ComponentMainThreadExecutorServiceAdapter.forMainThread()) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()) .setPartitionTracker(partitionTracker) .build(); @@ -107,8 +117,10 @@ public void testFindLatestAllocationIgnoresFailedAttempts() throws Exception { final TestingPhysicalSlotProvider withLimitedAmountOfPhysicalSlots = TestingPhysicalSlotProvider.createWithLimitedAmountOfPhysicalSlots(1); final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( - jobGraph, ComponentMainThreadExecutorServiceAdapter.forMainThread()) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( withLimitedAmountOfPhysicalSlots)) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FinalizeOnMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FinalizeOnMasterTest.java index 29594c369e1ab..3524cc1481763 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FinalizeOnMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/FinalizeOnMasterTest.java @@ -24,10 +24,15 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.scheduler.SchedulerBase; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; import org.junit.Test; +import java.util.concurrent.ScheduledExecutorService; + import static org.apache.flink.runtime.scheduler.SchedulerTestingUtils.createScheduler; import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.any; @@ -41,6 +46,10 @@ */ public class FinalizeOnMasterTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Test public void testFinalizeIsCalledUponSuccess() throws Exception { final JobVertex vertex1 = spy(new JobVertex("test vertex 1")); @@ -54,7 +63,8 @@ public void testFinalizeIsCalledUponSuccess() throws Exception { final SchedulerBase scheduler = createScheduler( JobGraphTestUtils.streamingJobGraph(vertex1, vertex2), - ComponentMainThreadExecutorServiceAdapter.forMainThread()); + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()); scheduler.startScheduling(); final ExecutionGraph eg = scheduler.getExecutionGraph(); @@ -82,7 +92,8 @@ public void testFinalizeIsNotCalledUponFailure() throws Exception { final SchedulerBase scheduler = createScheduler( JobGraphTestUtils.streamingJobGraph(vertex), - ComponentMainThreadExecutorServiceAdapter.forMainThread()); + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()); scheduler.startScheduling(); final ExecutionGraph eg = scheduler.getExecutionGraph(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartitionTest.java index 56be6d2f167b5..e5470cb99cde3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartitionTest.java @@ -33,8 +33,11 @@ import org.apache.flink.runtime.scheduler.strategy.ConsumedPartitionGroup; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.DirectScheduledExecutorService; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; import org.junit.Test; import java.util.Arrays; @@ -52,6 +55,9 @@ /** Tests for {@link IntermediateResultPartition}. */ public class IntermediateResultPartitionTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); @Test public void testPipelinedPartitionConsumable() throws Exception { @@ -202,7 +208,8 @@ private void testGetNumberOfSubpartitions( consumerParallelism, consumerMaxParallelism, distributionPattern, - isDynamicGraph); + isDynamicGraph, + EXECUTOR_RESOURCE.getExecutor()); final Iterator vertexIterator = eg.getVerticesTopologically().iterator(); @@ -227,7 +234,8 @@ public static ExecutionGraph createExecutionGraph( int consumerParallelism, int consumerMaxParallelism, DistributionPattern distributionPattern, - boolean isDynamicGraph) + boolean isDynamicGraph, + ScheduledExecutorService scheduledExecutorService) throws Exception { final JobVertex v1 = new JobVertex("v1"); @@ -262,9 +270,9 @@ public static ExecutionGraph createExecutionGraph( isDynamicGraph, consumerMaxParallelism)); if (isDynamicGraph) { - return builder.buildDynamicGraph(); + return builder.buildDynamicGraph(scheduledExecutorService); } else { - return builder.build(); + return builder.build(scheduledExecutorService); } } @@ -296,10 +304,10 @@ private static IntermediateResult createResult( JobGraph jobGraph = JobGraphTestUtils.batchJobGraph(source, sink); SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( - jobGraph, ComponentMainThreadExecutorServiceAdapter.forMainThread()) - .setIoExecutor(executorService) - .setFutureExecutor(executorService) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + executorService) .build(); ExecutionJobVertex ejv = scheduler.getExecutionJobVertex(source.getID()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java index f9712b88923c9..d471cac83f54d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java @@ -26,12 +26,16 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.scheduler.SchedulerBase; import org.apache.flink.runtime.scheduler.strategy.ConsumedPartitionGroup; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; +import org.junit.ClassRule; import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.concurrent.ScheduledExecutorService; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -42,6 +46,9 @@ * EdgeManagerBuildUtil#connectVertexToResult}. */ public class PointwisePatternTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); @Test public void testNToN() throws Exception { @@ -256,7 +263,7 @@ private ExecutionJobVertex setUpExecutionGraphAndGetDownstreamVertex( TestingDefaultExecutionGraphBuilder.newBuilder() .setVertexParallelismStore( SchedulerBase.computeVertexParallelismStore(ordered)) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); try { eg.attachJobGraph(ordered); } catch (JobException e) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/RemoveCachedShuffleDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/RemoveCachedShuffleDescriptorTest.java index 086dbec10c47a..f81c7bc7f1db7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/RemoveCachedShuffleDescriptorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/RemoveCachedShuffleDescriptorTest.java @@ -42,10 +42,13 @@ import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import java.util.ArrayList; @@ -71,6 +74,10 @@ public class RemoveCachedShuffleDescriptorTest extends TestLogger { private static final int PARALLELISM = 4; + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private ScheduledExecutorService scheduledExecutorService; private ComponentMainThreadExecutor mainThreadExecutor; private ManuallyTriggeredScheduledExecutorService ioExecutor; @@ -386,7 +393,8 @@ private static DefaultScheduler createScheduler( .addJobVertices(jobVertices) .build(); - return SchedulerTestingUtils.newSchedulerBuilder(jobGraph, mainThreadExecutor) + return new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, mainThreadExecutor, EXECUTOR_RESOURCE.getExecutor()) .setRestartBackoffTimeStrategy(new TestRestartBackoffTimeStrategy(true, 0)) .setBlobWriter(blobWriter) .setIoExecutor(ioExecutor) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingDefaultExecutionGraphBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingDefaultExecutionGraphBuilder.java index a5fc6d192527d..8e5e57ddea022 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingDefaultExecutionGraphBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TestingDefaultExecutionGraphBuilder.java @@ -41,13 +41,11 @@ import org.apache.flink.runtime.scheduler.VertexParallelismStore; import org.apache.flink.runtime.shuffle.ShuffleMaster; import org.apache.flink.runtime.shuffle.ShuffleTestUtils; -import org.apache.flink.testutils.TestingUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Optional; -import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; /** Builder of {@link ExecutionGraph} used in testing. */ @@ -60,8 +58,6 @@ public static TestingDefaultExecutionGraphBuilder newBuilder() { return new TestingDefaultExecutionGraphBuilder(); } - private ScheduledExecutorService futureExecutor = TestingUtils.defaultExecutor(); - private Executor ioExecutor = TestingUtils.defaultExecutor(); private Time rpcTimeout = Time.fromDuration(AkkaOptions.ASK_TIMEOUT_DURATION.defaultValue()); private ClassLoader userClassLoader = DefaultExecutionGraph.class.getClassLoader(); private BlobWriter blobWriter = VoidBlobWriter.getInstance(); @@ -90,17 +86,6 @@ public TestingDefaultExecutionGraphBuilder setJobGraph(JobGraph jobGraph) { return this; } - public TestingDefaultExecutionGraphBuilder setFutureExecutor( - ScheduledExecutorService futureExecutor) { - this.futureExecutor = futureExecutor; - return this; - } - - public TestingDefaultExecutionGraphBuilder setIoExecutor(Executor ioExecutor) { - this.ioExecutor = ioExecutor; - return this; - } - public TestingDefaultExecutionGraphBuilder setRpcTimeout(Time rpcTimeout) { this.rpcTimeout = rpcTimeout; return this; @@ -157,13 +142,14 @@ public TestingDefaultExecutionGraphBuilder setVertexParallelismStore( return this; } - private DefaultExecutionGraph build(boolean isDynamicGraph) + private DefaultExecutionGraph build( + boolean isDynamicGraph, ScheduledExecutorService executorService) throws JobException, JobExecutionException { return DefaultExecutionGraphBuilder.buildGraph( jobGraph, jobMasterConfig, - futureExecutor, - ioExecutor, + executorService, + executorService, userClassLoader, completedCheckpointStore, new CheckpointsCleaner(), @@ -185,11 +171,13 @@ private DefaultExecutionGraph build(boolean isDynamicGraph) isDynamicGraph); } - public DefaultExecutionGraph build() throws JobException, JobExecutionException { - return build(false); + public DefaultExecutionGraph build(ScheduledExecutorService executorService) + throws JobException, JobExecutionException { + return build(false, executorService); } - public DefaultExecutionGraph buildDynamicGraph() throws JobException, JobExecutionException { - return build(true); + public DefaultExecutionGraph buildDynamicGraph(ScheduledExecutorService executorService) + throws JobException, JobExecutionException { + return build(true, executorService); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java index 3c56d351bf93b..19c2ecc8c9340 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java @@ -24,12 +24,16 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.scheduler.SchedulerBase; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; +import org.junit.ClassRule; import org.junit.Test; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.concurrent.ScheduledExecutorService; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; @@ -39,6 +43,9 @@ /** Tests creating and initializing {@link SlotSharingGroup}. */ public class VertexSlotSharingTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); /** * Test setup: - v1 is isolated, no slot sharing. - v2 and v3 (not connected) share slots. - v4 @@ -84,7 +91,7 @@ public void testAssignSlotSharingGroup() { TestingDefaultExecutionGraphBuilder.newBuilder() .setVertexParallelismStore( SchedulerBase.computeVertexParallelismStore(vertices)) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); eg.attachJobGraph(vertices); // verify that the vertices are all in the same slot sharing group 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 eee04df028e93..955338f9f3ee8 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 @@ -23,12 +23,14 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.rpc.exceptions.RecipientUnreachableException; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; import org.hamcrest.Matcher; +import org.junit.ClassRule; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,6 +44,7 @@ import java.util.Queue; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -59,6 +62,11 @@ /** Tests for the {@link HeartbeatManager}. */ public class HeartbeatManagerTest extends TestLogger { + + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static final Logger LOG = LoggerFactory.getLogger(HeartbeatManagerTest.class); public static final long HEARTBEAT_INTERVAL = 50L; public static final long HEARTBEAT_TIMEOUT = 200L; @@ -89,7 +97,7 @@ public void testRegularHeartbeat() throws InterruptedException { FAILED_RPC_THRESHOLD, ownResourceID, heartbeatListener, - TestingUtils.defaultScheduledExecutor(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()), LOG); final ArrayBlockingQueue reportedPayloadsHeartbeatTarget = @@ -176,7 +184,7 @@ public void testHeartbeatTimeout() throws Exception { FAILED_RPC_THRESHOLD, ownResourceID, heartbeatListener, - TestingUtils.defaultScheduledExecutor(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()), LOG); final HeartbeatTarget heartbeatTarget = @@ -238,7 +246,7 @@ public void testHeartbeatCluster() throws Exception { FAILED_RPC_THRESHOLD, resourceIdTarget, heartbeatListenerTarget, - TestingUtils.defaultScheduledExecutor(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()), LOG); HeartbeatManagerSenderImpl heartbeatManagerSender = @@ -248,7 +256,7 @@ public void testHeartbeatCluster() throws Exception { FAILED_RPC_THRESHOLD, resourceIDSender, heartbeatListenerSender, - TestingUtils.defaultScheduledExecutor(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()), LOG); heartbeatManagerTarget.monitorTarget(resourceIDSender, heartbeatManagerSender); @@ -294,7 +302,7 @@ public void testTargetUnmonitoring() throws Exception { FAILED_RPC_THRESHOLD, resourceID, heartbeatListener, - TestingUtils.defaultScheduledExecutor(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()), LOG); final HeartbeatTarget heartbeatTarget = @@ -325,7 +333,7 @@ public void testLastHeartbeatFromUnregisteredTarget() { FAILED_RPC_THRESHOLD, resourceId, heartbeatListener, - TestingUtils.defaultScheduledExecutor(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()), LOG); try { @@ -348,7 +356,7 @@ public void testLastHeartbeatFrom() { FAILED_RPC_THRESHOLD, resourceId, new TestingHeartbeatListenerBuilder<>().createNewTestingHeartbeatListener(), - TestingUtils.defaultScheduledExecutor(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()), LOG); try { @@ -413,7 +421,7 @@ public void testHeartbeatManagerTargetPayload() throws Exception { FAILED_RPC_THRESHOLD, ResourceID.generate(), testingHeartbeatListener, - TestingUtils.defaultScheduledExecutor(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()), LOG); try { @@ -511,7 +519,7 @@ public void testHeartbeatManagerSenderMarksTargetUnreachableOnRecipientUnreachab 1, ResourceID.generate(), testingHeartbeatListener, - TestingUtils.defaultScheduledExecutor(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()), LOG); try { @@ -551,7 +559,7 @@ public void testHeartbeatManagerMarksTargetUnreachableOnRecipientUnreachableExce failedRpcRequestsUntilUnreachable, ResourceID.generate(), testingHeartbeatListener, - TestingUtils.defaultScheduledExecutor(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()), LOG); try { @@ -601,7 +609,7 @@ public void testHeartbeatManagerIgnoresRecipientUnreachableExceptionIfDisabled() -1, // disable rpc request checking ResourceID.generate(), testingHeartbeatListener, - TestingUtils.defaultScheduledExecutor(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()), LOG); try { @@ -650,7 +658,7 @@ public void testHeartbeatManagerResetsFailedRpcCountOnSuccessfulRpc() throws Exc 3, ResourceID.generate(), testingHeartbeatListener, - TestingUtils.defaultScheduledExecutor(), + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()), LOG); try { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHAJobRunITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHAJobRunITCase.java index 80f9407386564..941cb55f9237a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHAJobRunITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHAJobRunITCase.java @@ -31,8 +31,10 @@ import org.apache.flink.runtime.zookeeper.ZooKeeperExtension; import org.apache.flink.test.junit5.InjectMiniCluster; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorExtension; import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; import org.junit.jupiter.api.Order; import org.junit.jupiter.api.Test; @@ -40,6 +42,7 @@ import org.junit.jupiter.api.extension.RegisterExtension; import java.time.Duration; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import static org.assertj.core.api.Assertions.assertThat; @@ -54,6 +57,10 @@ @ExtendWith(TestLoggerExtension.class) public abstract class AbstractHAJobRunITCase { + @RegisterExtension + static final TestExecutorExtension EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorExtension(); + @RegisterExtension @Order(1) private static final AllCallbackWrapper ZOOKEEPER_EXTENSION = @@ -92,7 +99,8 @@ public void testJobExecutionInHaMode(@InjectMiniCluster MiniCluster flinkCluster Time.milliseconds(10), deadline, status -> flinkCluster.isRunning() && status == JobStatus.FINISHED, - TestingUtils.defaultScheduledExecutor()) + new ScheduledExecutorServiceAdapter( + EXECUTOR_RESOURCE.getExecutor())) .get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); assertThat(jobStatus).isEqualTo(JobStatus.FINISHED); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderServiceTest.java index 61365adf2a7f9..4911f350b11f1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedLeaderServiceTest.java @@ -20,12 +20,15 @@ import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -34,6 +37,9 @@ /** Tests for the {@link EmbeddedLeaderService}. */ public class EmbeddedLeaderServiceTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); /** * Tests that the {@link EmbeddedLeaderService} can handle a concurrent grant leadership call * and a shutdown. @@ -41,7 +47,7 @@ public class EmbeddedLeaderServiceTest extends TestLogger { @Test public void testConcurrentGrantLeadershipAndShutdown() throws Exception { final EmbeddedLeaderService embeddedLeaderService = - new EmbeddedLeaderService(TestingUtils.defaultExecutor()); + new EmbeddedLeaderService(EXECUTOR_RESOURCE.getExecutor()); try { final LeaderElectionService leaderElectionService = @@ -73,7 +79,7 @@ public void testConcurrentGrantLeadershipAndShutdown() throws Exception { @Test public void testConcurrentRevokeLeadershipAndShutdown() throws Exception { final EmbeddedLeaderService embeddedLeaderService = - new EmbeddedLeaderService(TestingUtils.defaultExecutor()); + new EmbeddedLeaderService(EXECUTOR_RESOURCE.getExecutor()); try { final LeaderElectionService leaderElectionService = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java index 71d1e5f2a1ce6..b45f142ffef09 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannelTest.java @@ -57,12 +57,15 @@ import org.apache.flink.runtime.taskexecutor.PartitionProducerStateChecker; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TestTaskBuilder; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import javax.annotation.Nullable; @@ -80,6 +83,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; @@ -116,6 +120,9 @@ /** Tests for the {@link RemoteInputChannel}. */ public class RemoteInputChannelTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); private static final long CHECKPOINT_ID = 1L; private static final CheckpointOptions UNALIGNED = @@ -1797,7 +1804,7 @@ public void testOnFailedPartitionRequestDoesNotBlockNetworkThreads() throws Exce final Task task = new TestTaskBuilder(shuffleEnvironment) .setPartitionProducerStateChecker(partitionProducerStateChecker) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final SingleInputGate inputGate = new SingleInputGateBuilder().setPartitionProducerStateProvider(task).build(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/UpdatePartitionConsumersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/UpdatePartitionConsumersTest.java index ee58347edc22b..b1b46dee1a0d2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/UpdatePartitionConsumersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/UpdatePartitionConsumersTest.java @@ -38,14 +38,18 @@ import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; import org.apache.flink.runtime.shuffle.UnknownShuffleDescriptor; import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.IterableUtils; import org.apache.flink.util.TestLogger; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -58,6 +62,10 @@ /** Tests for the updating of consumers depending on the producers result. */ public class UpdatePartitionConsumersTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static final long TIMEOUT = 5000L; private JobGraph jobGraph; @@ -124,8 +132,10 @@ public void testUpdatePartitionConsumers() throws Exception { new SimpleAckingTaskManagerGateway(); final SchedulerBase scheduler = - SchedulerTestingUtils.newSchedulerBuilder( - jobGraph, ComponentMainThreadExecutorServiceAdapter.forMainThread()) + new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( new TestExecutionSlotAllocatorFactory(taskManagerGateway)) .build(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java index 03a6733b3dbc2..51b49fb4a1c17 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterPartitionReleaseTest.java @@ -250,8 +250,6 @@ public TestSetup( new JobMasterBuilder(jobGraph, rpcService) .withConfiguration(configuration) .withHighAvailabilityServices(haServices) - .withJobManagerSharedServices( - new TestingJobManagerSharedServicesBuilder().build()) .withFatalErrorHandler(fatalErrorHandler) .withHeartbeatServices(heartbeatServices) .withPartitionTrackerFactory(ignored -> partitionTracker) 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 1ac904e49b6f3..a6ff8e935f7cf 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 @@ -1550,7 +1550,6 @@ public void testTaskExecutorNotReleasedOnFailedAllocationIfPartitionIsAllocated( new JobMasterBuilder(jobGraph, rpcService) .withConfiguration(configuration) .withHighAvailabilityServices(haServices) - .withJobManagerSharedServices(jobManagerSharedServices) .withHeartbeatServices(heartbeatServices) .withPartitionTrackerFactory(ignored -> partitionTracker) .createJobMaster()) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerSharedServicesBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerSharedServicesBuilder.java index 75b7cf2a365a5..a15bfe6ba2785 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerSharedServicesBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerSharedServicesBuilder.java @@ -24,17 +24,13 @@ import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.shuffle.ShuffleMaster; import org.apache.flink.runtime.shuffle.ShuffleTestUtils; -import org.apache.flink.testutils.TestingUtils; +import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; /** Builder for the {@link JobManagerSharedServices}. */ public class TestingJobManagerSharedServicesBuilder { - private ScheduledExecutorService futureExecutor; - - private ScheduledExecutorService ioExecutor; - private LibraryCacheManager libraryCacheManager; private ShuffleMaster shuffleMaster; @@ -42,24 +38,11 @@ public class TestingJobManagerSharedServicesBuilder { private BlobWriter blobWriter; public TestingJobManagerSharedServicesBuilder() { - futureExecutor = TestingUtils.defaultExecutor(); - ioExecutor = TestingUtils.defaultExecutor(); libraryCacheManager = ContextClassLoaderLibraryCacheManager.INSTANCE; shuffleMaster = ShuffleTestUtils.DEFAULT_SHUFFLE_MASTER; blobWriter = VoidBlobWriter.getInstance(); } - public TestingJobManagerSharedServicesBuilder futureExecutor( - ScheduledExecutorService futureExecutor) { - this.futureExecutor = futureExecutor; - return this; - } - - public TestingJobManagerSharedServicesBuilder ioExecutor(ScheduledExecutorService ioExecutor) { - this.ioExecutor = ioExecutor; - return this; - } - public TestingJobManagerSharedServicesBuilder setShuffleMaster(ShuffleMaster shuffleMaster) { this.shuffleMaster = shuffleMaster; return this; @@ -76,7 +59,10 @@ public void setBlobWriter(BlobWriter blobWriter) { } public JobManagerSharedServices build() { + final ScheduledExecutorService executorService = + Executors.newSingleThreadScheduledExecutor(); + return new JobManagerSharedServices( - futureExecutor, ioExecutor, libraryCacheManager, shuffleMaster, blobWriter); + executorService, executorService, libraryCacheManager, shuffleMaster, blobWriter); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/JobMasterBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/JobMasterBuilder.java index 45fc3d0021d10..8aebbc046ebe0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/JobMasterBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/utils/JobMasterBuilder.java @@ -65,8 +65,7 @@ public class JobMasterBuilder { private HighAvailabilityServices highAvailabilityServices; - private JobManagerSharedServices jobManagerSharedServices = - new TestingJobManagerSharedServicesBuilder().build(); + private JobManagerSharedServices jobManagerSharedServices = null; private HeartbeatServices heartbeatServices = DEFAULT_HEARTBEAT_SERVICES; @@ -188,7 +187,9 @@ public JobMaster createJobMaster() throws Exception { ? slotPoolServiceSchedulerFactory : DefaultSlotPoolServiceSchedulerFactory.fromConfiguration( configuration, jobGraph.getJobType()), - jobManagerSharedServices, + jobManagerSharedServices != null + ? jobManagerSharedServices + : new TestingJobManagerSharedServicesBuilder().build(), heartbeatServices, UnregisteredJobManagerJobMetricGroupFactory.INSTANCE, onCompletionActions, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java index f25b13069452f..90c1c1ad280c6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java @@ -34,15 +34,18 @@ import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.util.LeaderRetrievalUtils; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import java.time.Duration; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; @@ -58,6 +61,10 @@ public class LeaderChangeClusterComponentsTest extends TestLogger { private static final int NUM_TMS = 2; public static final int PARALLELISM = SLOTS_PER_TM * NUM_TMS; + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static TestingMiniCluster miniCluster; private static EmbeddedHaServicesWithLeadershipControl highAvailabilityServices; @@ -70,7 +77,7 @@ public class LeaderChangeClusterComponentsTest extends TestLogger { public static void setupClass() throws Exception { highAvailabilityServices = - new EmbeddedHaServicesWithLeadershipControl(TestingUtils.defaultExecutor()); + new EmbeddedHaServicesWithLeadershipControl(EXECUTOR_RESOURCE.getExecutor()); miniCluster = TestingMiniCluster.newBuilder( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionTest.java index 28cd2b32c3ee1..a5614a1518bbc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionTest.java @@ -26,11 +26,13 @@ import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource; import org.apache.flink.runtime.util.ZooKeeperUtils; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.curator.test.TestingServer; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -40,6 +42,7 @@ import java.util.Collection; import java.util.UUID; import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; @@ -48,6 +51,10 @@ @RunWith(Parameterized.class) public class LeaderElectionTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Rule public final TestingFatalErrorHandlerResource testingFatalErrorHandlerResource = new TestingFatalErrorHandlerResource(); @@ -218,7 +225,7 @@ private static final class EmbeddedServiceClass implements ServiceClass { @Override public void setup(FatalErrorHandler fatalErrorHandler) { - embeddedLeaderService = new EmbeddedLeaderService(TestingUtils.defaultExecutor()); + embeddedLeaderService = new EmbeddedLeaderService(EXECUTOR_RESOURCE.getExecutor()); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalTest.java index 4e34818fb2435..5978fe78924d1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalTest.java @@ -32,11 +32,13 @@ import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource; import org.apache.flink.runtime.util.ZooKeeperUtils; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.curator.test.TestingServer; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; @@ -48,6 +50,7 @@ import java.net.SocketAddress; import java.net.UnknownHostException; import java.time.Duration; +import java.util.concurrent.ScheduledExecutorService; import static org.junit.Assert.assertEquals; @@ -56,6 +59,10 @@ public class ZooKeeperLeaderRetrievalTest extends TestLogger { private static final RpcSystem RPC_SYSTEM = RpcSystem.load(); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private TestingServer testingServer; private Configuration config; @@ -79,7 +86,7 @@ public void before() throws Exception { new ZooKeeperHaServices( ZooKeeperUtils.startCuratorFramework( config, testingFatalErrorHandlerResource.getFatalErrorHandler()), - TestingUtils.defaultExecutor(), + EXECUTOR_RESOURCE.getExecutor(), config, new VoidBlobStore()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorSchedulerTest.java index 1d8b59d7bf775..e8022c97ada58 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorSchedulerTest.java @@ -60,6 +60,8 @@ import org.apache.flink.runtime.state.ttl.TtlTimeProvider; import org.apache.flink.runtime.taskexecutor.TaskExecutorOperatorEventGateway; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.SerializedValue; @@ -68,6 +70,7 @@ import org.hamcrest.Matcher; import org.junit.After; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Test; @@ -111,6 +114,10 @@ public class OperatorCoordinatorSchedulerTest extends TestLogger { private final JobVertexID testVertexId = new JobVertexID(); private final OperatorID testOperatorId = new OperatorID(); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private final ManuallyTriggeredScheduledExecutorService executor = new ManuallyTriggeredScheduledExecutorService(); @@ -689,9 +696,13 @@ private DefaultScheduler setupTestJobAndScheduler( final SchedulerTestingUtils.DefaultSchedulerBuilder schedulerBuilder = taskExecutorOperatorEventGateway == null - ? SchedulerTestingUtils.createSchedulerBuilder(jobGraph, mainThreadExecutor) + ? SchedulerTestingUtils.createSchedulerBuilder( + jobGraph, mainThreadExecutor, EXECUTOR_RESOURCE.getExecutor()) : SchedulerTestingUtils.createSchedulerBuilder( - jobGraph, mainThreadExecutor, taskExecutorOperatorEventGateway); + jobGraph, + mainThreadExecutor, + taskExecutorOperatorEventGateway, + EXECUTOR_RESOURCE.getExecutor()); if (restartAllOnFailover) { schedulerBuilder.setFailoverStrategyFactory(new RestartAllFailoverStrategy.Factory()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java index 39480dfd31450..9d1bd05fabc42 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java @@ -21,6 +21,7 @@ import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; @@ -28,6 +29,7 @@ import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.slf4j.LoggerFactory; @@ -61,6 +63,10 @@ */ public class RetryingRegistrationTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private TestingRpcService rpcService; @Before @@ -147,7 +153,8 @@ public void testRetryConnectOnFailure() throws Exception { final String testId = "laissez les bon temps roulez"; final UUID leaderId = UUID.randomUUID(); - ScheduledExecutorService executor = TestingUtils.defaultExecutor(); + ScheduledExecutorServiceAdapter executor = + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()); ManualResponseTestRegistrationGateway testGateway = new ManualResponseTestRegistrationGateway(new TestRegistrationSuccess(testId)); @@ -163,16 +170,14 @@ public void testRetryConnectOnFailure() throws Exception { CompletableFuture.completedFuture( testGateway) // second connection attempt succeeds ); - when(rpc.getScheduledExecutor()) - .thenReturn(new ScheduledExecutorServiceAdapter(executor)); + when(rpc.getScheduledExecutor()).thenReturn(executor); when(rpc.scheduleRunnable(any(Runnable.class), anyLong(), any(TimeUnit.class))) .thenAnswer( (InvocationOnMock invocation) -> { final Runnable runnable = invocation.getArgument(0); final long delay = invocation.getArgument(1); final TimeUnit timeUnit = invocation.getArgument(2); - return TestingUtils.defaultScheduledExecutor() - .schedule(runnable, delay, timeUnit); + return executor.schedule(runnable, delay, timeUnit); }); TestRetryingRegistration registration = 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 a282ed3ea05a1..c5c87c49b408f 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 @@ -38,6 +38,7 @@ import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; @@ -46,12 +47,14 @@ import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import java.util.Collection; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; @@ -74,6 +77,10 @@ public class ResourceManagerTaskExecutorTest extends TestLogger { private static final ResourceProfile DEFAULT_SLOT_PROFILE = ResourceProfile.fromResources(1.0, 1234); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static TestingRpcService rpcService; private TestingTaskExecutorGateway taskExecutorGateway; @@ -219,7 +226,7 @@ public void testDelayedRegisterTaskExecutor() throws Exception { } return rpcGateway; }, - TestingUtils.defaultExecutor())); + EXECUTOR_RESOURCE.getExecutor())); TaskExecutorRegistration taskExecutorRegistration = new TaskExecutorRegistration( 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 8d5446c37bf79..0ebf21c594899 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 @@ -640,8 +640,7 @@ private TestingResourceManager buildAndStart() throws Exception { if (slotManager == null) { slotManager = - DeclarativeSlotManagerBuilder.newBuilder() - .setScheduledExecutor(rpcService.getScheduledExecutor()) + DeclarativeSlotManagerBuilder.newBuilder(rpcService.getScheduledExecutor()) .build(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerBuilder.java index 6492e06a0479b..4e6398456f95f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerBuilder.java @@ -35,7 +35,7 @@ /** Builder for {@link DeclarativeSlotManager}. */ public class DeclarativeSlotManagerBuilder { private SlotMatchingStrategy slotMatchingStrategy; - private ScheduledExecutor scheduledExecutor; + private final ScheduledExecutor scheduledExecutor; private Time taskManagerRequestTimeout; private Time slotRequestTimeout; private Time taskManagerTimeout; @@ -48,9 +48,9 @@ public class DeclarativeSlotManagerBuilder { private ResourceTracker resourceTracker; private SlotTracker slotTracker; - private DeclarativeSlotManagerBuilder() { + private DeclarativeSlotManagerBuilder(ScheduledExecutor scheduledExecutor) { this.slotMatchingStrategy = AnyMatchingSlotMatchingStrategy.INSTANCE; - this.scheduledExecutor = TestingUtils.defaultScheduledExecutor(); + this.scheduledExecutor = scheduledExecutor; this.taskManagerRequestTimeout = TestingUtils.infiniteTime(); this.slotRequestTimeout = TestingUtils.infiniteTime(); this.taskManagerTimeout = TestingUtils.infiniteTime(); @@ -66,13 +66,8 @@ private DeclarativeSlotManagerBuilder() { this.slotTracker = new DefaultSlotTracker(); } - public static DeclarativeSlotManagerBuilder newBuilder() { - return new DeclarativeSlotManagerBuilder(); - } - - public DeclarativeSlotManagerBuilder setScheduledExecutor(ScheduledExecutor scheduledExecutor) { - this.scheduledExecutor = scheduledExecutor; - return this; + public static DeclarativeSlotManagerBuilder newBuilder(ScheduledExecutor scheduledExecutor) { + return new DeclarativeSlotManagerBuilder(scheduledExecutor); } public DeclarativeSlotManagerBuilder setTaskManagerRequestTimeout( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java index fe5a7b692af86..5eb6cf7a264da 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DeclarativeSlotManagerTest.java @@ -46,16 +46,19 @@ import org.apache.flink.runtime.taskexecutor.exceptions.SlotOccupiedException; import org.apache.flink.runtime.testutils.SystemExitTrackingSecurityManager; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.util.concurrent.ScheduledExecutor; +import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; import org.apache.flink.util.function.FunctionUtils; import org.apache.flink.util.function.ThrowingConsumer; import org.apache.flink.shaded.guava30.com.google.common.collect.Iterators; +import org.junit.ClassRule; import org.junit.Test; import java.util.ArrayList; @@ -70,6 +73,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; @@ -93,6 +97,10 @@ /** Tests for the {@link DeclarativeSlotManager}. */ public class DeclarativeSlotManagerTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static final FlinkException TEST_EXCEPTION = new FlinkException("Test exception"); private static final WorkerResourceSpec WORKER_RESOURCE_SPEC = @@ -616,7 +624,7 @@ public void testSlotAllocationTimeout() throws Exception { final SlotReport slotReport = createSlotReport(taskManagerConnection.getResourceID(), 2); - final Executor mainThreadExecutor = TestingUtils.defaultExecutor(); + final Executor mainThreadExecutor = EXECUTOR_RESOURCE.getExecutor(); try (DeclarativeSlotManager slotManager = createDeclarativeSlotManagerBuilder().build()) { @@ -752,9 +760,7 @@ public void testSlotReportWithConflictingJobIdDuringSlotAllocation() throws Exce final ScheduledExecutor mainThreadExecutor = new ManuallyTriggeredScheduledExecutor(); try (final DeclarativeSlotManager slotManager = - createDeclarativeSlotManagerBuilder() - .setScheduledExecutor(mainThreadExecutor) - .build()) { + createDeclarativeSlotManagerBuilder(mainThreadExecutor).build()) { slotManager.start( ResourceManagerId.generate(), @@ -1526,14 +1532,21 @@ private DeclarativeSlotManager createSlotManager( ResourceManagerId resourceManagerId, ResourceActions resourceManagerActions, int numSlotsPerWorker) { - return createDeclarativeSlotManagerBuilder() + return createDeclarativeSlotManagerBuilder( + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor())) .setNumSlotsPerWorker(numSlotsPerWorker) .setRedundantTaskManagerNum(0) .buildAndStartWithDirectExec(resourceManagerId, resourceManagerActions); } private static DeclarativeSlotManagerBuilder createDeclarativeSlotManagerBuilder() { - return DeclarativeSlotManagerBuilder.newBuilder() + return createDeclarativeSlotManagerBuilder( + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor())); + } + + private static DeclarativeSlotManagerBuilder createDeclarativeSlotManagerBuilder( + ScheduledExecutor executor) { + return DeclarativeSlotManagerBuilder.newBuilder(executor) .setDefaultWorkerResourceSpec(WORKER_RESOURCE_SPEC); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotStatusSyncerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotStatusSyncerTest.java index cdd765638baa8..7d37cb884fbcf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotStatusSyncerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/DefaultSlotStatusSyncerTest.java @@ -33,13 +33,16 @@ import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; +import org.junit.ClassRule; import org.junit.Test; import java.util.Arrays; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeoutException; import static org.hamcrest.Matchers.contains; @@ -59,6 +62,10 @@ public class DefaultSlotStatusSyncerTest extends TestLogger { ResourceID.generate(), new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway()); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Test public void testAllocateSlot() throws Exception { final FineGrainedTaskManagerTracker taskManagerTracker = @@ -93,7 +100,7 @@ public void testAllocateSlot() throws Exception { taskManagerTracker, resourceTracker, ResourceManagerId.generate(), - TestingUtils.defaultExecutor()); + EXECUTOR_RESOURCE.getExecutor()); final CompletableFuture allocatedFuture = slotStatusSyncer.allocateSlot( @@ -140,7 +147,7 @@ public void testAllocateSlotFailsWithException() { taskManagerTracker, resourceTracker, ResourceManagerId.generate(), - TestingUtils.defaultExecutor()); + EXECUTOR_RESOURCE.getExecutor()); final CompletableFuture allocatedFuture = slotStatusSyncer.allocateSlot( @@ -176,7 +183,7 @@ public void testFreeSlot() { taskManagerTracker, resourceTracker, ResourceManagerId.generate(), - TestingUtils.defaultExecutor()); + EXECUTOR_RESOURCE.getExecutor()); taskManagerTracker.addTaskManager( TASK_EXECUTOR_CONNECTION, ResourceProfile.ANY, ResourceProfile.ANY); taskManagerTracker.notifySlotStatus( @@ -209,7 +216,7 @@ public void testSlotStatusProcessing() { taskManagerTracker, resourceTracker, ResourceManagerId.generate(), - TestingUtils.defaultExecutor()); + EXECUTOR_RESOURCE.getExecutor()); final TestingTaskExecutorGateway taskExecutorGateway = new TestingTaskExecutorGatewayBuilder() .setRequestSlotFunction(ignored -> new CompletableFuture<>()) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java index b1b73a53d40bc..1013f8919cc49 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/FineGrainedSlotManagerTestBase.java @@ -39,6 +39,7 @@ import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.concurrent.ScheduledExecutor; +import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; import org.apache.flink.util.function.RunnableWithException; import org.junit.ClassRule; @@ -48,8 +49,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -58,8 +58,8 @@ /** Base class for the tests of {@link FineGrainedSlotManager}. */ public abstract class FineGrainedSlotManagerTestBase extends TestLogger { @ClassRule - public static final TestExecutorResource EXECUTOR_RESOURCE = - new TestExecutorResource<>(() -> Executors.newSingleThreadExecutor()); + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); private static final long FUTURE_TIMEOUT_SECOND = 5; private static final long FUTURE_EXPECT_TIMEOUT_MS = 50; @@ -146,7 +146,8 @@ protected class Context { new DefaultSlotStatusSyncer(Time.seconds(10L)); private SlotManagerMetricGroup slotManagerMetricGroup = UnregisteredMetricGroups.createUnregisteredSlotManagerMetricGroup(); - private final ScheduledExecutor scheduledExecutor = TestingUtils.defaultScheduledExecutor(); + private final ScheduledExecutor scheduledExecutor = + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()); private final Executor mainThreadExecutor = EXECUTOR_RESOURCE.getExecutor(); private FineGrainedSlotManager slotManager; private long requirementCheckDelay = 0; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorManagerBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorManagerBuilder.java index 158387e118fcf..157e894aea1eb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorManagerBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorManagerBuilder.java @@ -19,7 +19,6 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.resourcemanager.WorkerResourceSpec; -import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.concurrent.Executors; import org.apache.flink.util.concurrent.ScheduledExecutor; @@ -34,10 +33,14 @@ public class TaskExecutorManagerBuilder { private boolean waitResultConsumedBeforeRelease = true; private int redundantTaskManagerNum = 0; private Time taskManagerTimeout = Time.seconds(5); - private ScheduledExecutor scheduledExecutor = TestingUtils.defaultScheduledExecutor(); + private final ScheduledExecutor scheduledExecutor; private Executor mainThreadExecutor = Executors.directExecutor(); private ResourceActions newResourceActions = new TestingResourceActionsBuilder().build(); + public TaskExecutorManagerBuilder(ScheduledExecutor scheduledExecutor) { + this.scheduledExecutor = scheduledExecutor; + } + public TaskExecutorManagerBuilder setDefaultWorkerResourceSpec( WorkerResourceSpec defaultWorkerResourceSpec) { this.defaultWorkerResourceSpec = defaultWorkerResourceSpec; @@ -70,11 +73,6 @@ public TaskExecutorManagerBuilder setTaskManagerTimeout(Time taskManagerTimeout) return this; } - public TaskExecutorManagerBuilder setScheduledExecutor(ScheduledExecutor scheduledExecutor) { - this.scheduledExecutor = scheduledExecutor; - return this; - } - public TaskExecutorManagerBuilder setMainThreadExecutor(Executor mainThreadExecutor) { this.mainThreadExecutor = mainThreadExecutor; return this; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorManagerTest.java index d221929297dbc..28593e31f7621 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/slotmanager/TaskExecutorManagerTest.java @@ -30,13 +30,17 @@ import org.apache.flink.runtime.taskexecutor.SlotStatus; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; +import org.junit.ClassRule; import org.junit.Test; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -50,6 +54,10 @@ /** Tests for the {@link TaskExecutorManager}. */ public class TaskExecutorManagerTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + /** Tests that a pending slot is only fulfilled by an exactly matching received slot. */ @Test public void testPendingSlotNotFulfilledIfProfilesAreNotExactMatch() { @@ -141,7 +149,7 @@ public void testTaskManagerTimeoutDoesNotRemoveSlots() throws Exception { (instanceId, ignored) -> releaseResourceFuture.complete(instanceId)) .build(); - final Executor mainThreadExecutor = TestingUtils.defaultExecutor(); + final Executor mainThreadExecutor = EXECUTOR_RESOURCE.getExecutor(); try (final TaskExecutorManager taskExecutorManager = createTaskExecutorManagerBuilder() @@ -193,7 +201,7 @@ public void testTimeoutForUnusedTaskManager() throws Exception { (instanceID, e) -> releaseResourceFuture.complete(instanceID)) .build(); - final Executor mainThreadExecutor = TestingUtils.defaultExecutor(); + final Executor mainThreadExecutor = EXECUTOR_RESOURCE.getExecutor(); try (final TaskExecutorManager taskExecutorManager = createTaskExecutorManagerBuilder() @@ -366,7 +374,8 @@ public void testGetResourceOverview() { } private static TaskExecutorManagerBuilder createTaskExecutorManagerBuilder() { - return new TaskExecutorManagerBuilder() + return new TaskExecutorManagerBuilder( + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor())) .setResourceActions(createResourceActionsBuilder().build()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/MockResourceManagerRuntimeServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/MockResourceManagerRuntimeServices.java index 14e92e67e4705..730d0a2dfddb3 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/MockResourceManagerRuntimeServices.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/utils/MockResourceManagerRuntimeServices.java @@ -51,8 +51,7 @@ public MockResourceManagerRuntimeServices(RpcService rpcService, Time timeout) { this( rpcService, timeout, - DeclarativeSlotManagerBuilder.newBuilder() - .setScheduledExecutor( + DeclarativeSlotManagerBuilder.newBuilder( new ScheduledExecutorServiceAdapter( new DirectScheduledExecutorService())) .setTaskManagerRequestTimeout(Time.seconds(10)) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestClientMultipartTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestClientMultipartTest.java index ea7b1fa02bda5..dbfbbd7b76955 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestClientMultipartTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestClientMultipartTest.java @@ -23,9 +23,9 @@ import org.apache.flink.runtime.rest.messages.EmptyMessageParameters; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.EmptyResponseBody; -import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.ConfigurationException; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.concurrent.Executors; import org.junit.After; import org.junit.AfterClass; @@ -50,7 +50,7 @@ public class RestClientMultipartTest extends TestLogger { @BeforeClass public static void setupClient() throws ConfigurationException { - restClient = new RestClient(new Configuration(), TestingUtils.defaultExecutor()); + restClient = new RestClient(new Configuration(), Executors.directExecutor()); } @After diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestClientTest.java index c76b2379c11dc..a29b535968525 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestClientTest.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.rest.messages.MessageHeaders; import org.apache.flink.runtime.rest.versioning.RestAPIVersion; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.NetUtils; import org.apache.flink.util.TestLogger; @@ -36,6 +37,7 @@ import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import java.io.IOException; @@ -44,6 +46,7 @@ import java.util.Collections; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -53,6 +56,9 @@ /** Tests for {@link RestClient}. */ public class RestClientTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); private static final String unroutableIp = "240.0.0.0"; @@ -104,7 +110,7 @@ public void testConnectionClosedHandling() throws Exception { config.setLong(RestOptions.IDLENESS_TIMEOUT, 5000L); try (final ServerSocket serverSocket = new ServerSocket(0); final RestClient restClient = - new RestClient(config, TestingUtils.defaultExecutor())) { + new RestClient(config, EXECUTOR_RESOURCE.getExecutor())) { final String targetAddress = "localhost"; final int targetPort = serverSocket.getLocalPort(); @@ -158,7 +164,7 @@ public void testRestClientClosedHandling() throws Exception { try (final ServerSocket serverSocket = new ServerSocket(0); final RestClient restClient = - new RestClient(config, TestingUtils.defaultExecutor())) { + new RestClient(config, EXECUTOR_RESOURCE.getExecutor())) { final String targetAddress = "localhost"; final int targetPort = serverSocket.getLocalPort(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestExternalHandlersITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestExternalHandlersITCase.java index 3c6f6b7789fbd..ae57a12c7fc6e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestExternalHandlersITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestExternalHandlersITCase.java @@ -31,8 +31,8 @@ import org.apache.flink.runtime.rest.messages.ResponseBody; import org.apache.flink.runtime.rest.util.TestRestServerEndpoint; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorExtension; import org.apache.flink.testutils.junit.extensions.ContextClassLoaderExtension; -import org.apache.flink.util.ConfigurationException; import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; @@ -47,6 +47,7 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -77,6 +78,10 @@ public class RestExternalHandlersITCase extends TestLogger { Prio1OutboundChannelHandlerFactory.class.getCanonicalName()) .build(); + @RegisterExtension + static final TestExecutorExtension EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorExtension(); + private final Configuration config; public RestExternalHandlersITCase() { @@ -99,7 +104,7 @@ private static Configuration getBaseConfig() { @BeforeEach private void setup() throws Exception { serverEndpoint = TestRestServerEndpoint.builder(config).buildAndStart(); - restClient = new TestRestClient(config); + restClient = new RestClient(config, EXECUTOR_RESOURCE.getExecutor()); serverAddress = serverEndpoint.getServerAddress(); } @@ -158,13 +163,6 @@ private CompletableFuture sendRequestToTestHandler( } } - static class TestRestClient extends RestClient { - - TestRestClient(Configuration configuration) throws ConfigurationException { - super(configuration, TestingUtils.defaultExecutor()); - } - } - private static class TestRequest implements RequestBody {} private static class TestResponse implements ResponseBody {} 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 a489b241e0d69..611b1978f6cd9 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 @@ -52,7 +52,7 @@ import org.apache.flink.runtime.webmonitor.TestingRestfulGateway; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; import org.apache.flink.testutils.TestingUtils; -import org.apache.flink.util.ConfigurationException; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.TestLogger; @@ -70,6 +70,7 @@ import org.junit.After; import org.junit.Assume; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -98,6 +99,7 @@ import java.util.Collections; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; @@ -126,6 +128,10 @@ public class RestServerEndpointITCase extends TestLogger { private static final Time timeout = Time.seconds(10L); private static final int TEST_REST_MAX_CONTENT_LENGTH = 4096; + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); private RestServerEndpoint serverEndpoint; @@ -241,7 +247,7 @@ public void setup() throws Exception { WebContentHandlerSpecification.getInstance(), staticFileServerHandler) .buildAndStart(); - restClient = new TestRestClient(config); + restClient = new RestClient(config, EXECUTOR_RESOURCE.getExecutor()); serverAddress = serverEndpoint.getServerAddress(); } @@ -780,13 +786,6 @@ private static TestParameters createTestParameters() { return parameters; } - static class TestRestClient extends RestClient { - - TestRestClient(Configuration configuration) throws ConfigurationException { - super(configuration, TestingUtils.defaultExecutor()); - } - } - private static class TestRequest implements RequestBody { public final int id; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerSSLAuthITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerSSLAuthITCase.java index 1b6fe807d71c0..eaad8caadb4e8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerSSLAuthITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/RestServerSSLAuthITCase.java @@ -33,6 +33,7 @@ import org.apache.flink.runtime.webmonitor.TestingRestfulGateway; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.concurrent.Executors; import org.junit.Test; import org.junit.runner.RunWith; @@ -114,7 +115,7 @@ public void testConnectFailure() throws Exception { TestRestServerEndpoint.builder(serverConfig) .withHandler(testVersionHandler.getMessageHeaders(), testVersionHandler) .buildAndStart(); - restClient = new RestServerEndpointITCase.TestRestClient(clientConfig); + restClient = new RestClient(clientConfig, Executors.directExecutor()); CompletableFuture response = restClient.sendRequest( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobConfigHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobConfigHandlerTest.java index 9a9233106c7ab..ce23fe11d571a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobConfigHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobConfigHandlerTest.java @@ -34,6 +34,7 @@ import org.apache.flink.runtime.rest.messages.JobMessageParameters; import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.concurrent.Executors; import org.junit.Test; @@ -58,7 +59,7 @@ public void handleRequest_executionConfigWithSecretValues_excludesSecretValuesFr Collections.emptyMap(), JobConfigHeaders.getInstance(), new DefaultExecutionGraphCache(TestingUtils.TIMEOUT, TestingUtils.TIMEOUT), - TestingUtils.defaultExecutor()); + Executors.directExecutor()); final Map globalJobParameters = new HashMap<>(); globalJobParameters.put("foobar", "barfoo"); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandlerTest.java index 2bfdcf209cd29..78943aa2ffa23 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/JobExceptionsHandlerTest.java @@ -52,6 +52,7 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.SerializedThrowable; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.concurrent.Executors; import org.hamcrest.Description; import org.hamcrest.Matcher; @@ -89,7 +90,7 @@ public class JobExceptionsHandlerTest extends TestLogger { Collections.emptyMap(), JobExceptionsHeaders.getInstance(), new DefaultExecutionGraphCache(TestingUtils.TIMEOUT, TestingUtils.TIMEOUT), - TestingUtils.defaultExecutor()); + Executors.directExecutor()); @Test public void testNoExceptions() throws HandlerRequestException { 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 d1830d266958f..3c9ca8ab6d88e 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 @@ -34,9 +34,9 @@ import org.apache.flink.runtime.rest.messages.job.JobSubmitRequestBody; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; -import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.concurrent.Executors; import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus; @@ -115,7 +115,7 @@ public void testSerializationFailureHandling() throws Exception { () -> CompletableFuture.completedFuture(mockGateway), RpcUtils.INF_TIMEOUT, Collections.emptyMap(), - TestingUtils.defaultExecutor(), + Executors.directExecutor(), configuration); JobSubmitRequestBody request = @@ -151,7 +151,7 @@ public void testSuccessfulJobSubmission() throws Exception { () -> CompletableFuture.completedFuture(mockGateway), RpcUtils.INF_TIMEOUT, Collections.emptyMap(), - TestingUtils.defaultExecutor(), + Executors.directExecutor(), configuration); JobSubmitRequestBody request = @@ -189,7 +189,7 @@ public void testRejectionOnCountMismatch() throws Exception { () -> CompletableFuture.completedFuture(mockGateway), RpcUtils.INF_TIMEOUT, Collections.emptyMap(), - TestingUtils.defaultExecutor(), + Executors.directExecutor(), configuration); JobSubmitRequestBody request = @@ -236,7 +236,7 @@ public void testFileHandling() throws Exception { () -> CompletableFuture.completedFuture(dispatcherGateway), RpcUtils.INF_TIMEOUT, Collections.emptyMap(), - TestingUtils.defaultExecutor(), + Executors.directExecutor(), configuration); final Path jobGraphFile = TEMPORARY_FOLDER.newFile().toPath(); @@ -294,7 +294,7 @@ public void testFailedJobSubmission() throws Exception { () -> CompletableFuture.completedFuture(mockGateway), RpcUtils.INF_TIMEOUT, Collections.emptyMap(), - TestingUtils.defaultExecutor(), + Executors.directExecutor(), configuration); final Path jobGraphFile = TEMPORARY_FOLDER.newFile().toPath(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskCurrentAttemptDetailsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskCurrentAttemptDetailsHandlerTest.java index e23dfdaae578d..bd60ce534e254 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskCurrentAttemptDetailsHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskCurrentAttemptDetailsHandlerTest.java @@ -44,8 +44,8 @@ import org.apache.flink.runtime.rest.messages.job.metrics.IOMetricsInfo; import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation; import org.apache.flink.runtime.util.EvictingBoundedList; -import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.concurrent.Executors; import org.junit.Test; @@ -114,7 +114,7 @@ public void testHandleRequest() throws Exception { new MetricFetcherImpl<>( () -> null, address -> null, - TestingUtils.defaultExecutor(), + Executors.directExecutor(), Time.milliseconds(1000L), MetricOptions.METRIC_FETCHER_UPDATE_INTERVAL.defaultValue()); @@ -127,7 +127,7 @@ public void testHandleRequest() throws Exception { new DefaultExecutionGraphCache( restHandlerConfiguration.getTimeout(), Time.milliseconds(restHandlerConfiguration.getRefreshInterval())), - TestingUtils.defaultExecutor(), + Executors.directExecutor(), metricFetcher); final HashMap receivedPathParameters = new HashMap<>(2); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptAccumulatorsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptAccumulatorsHandlerTest.java index 13910ddd3f46a..0b6b2ea26c0db 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptAccumulatorsHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptAccumulatorsHandlerTest.java @@ -35,10 +35,10 @@ import org.apache.flink.runtime.rest.messages.job.SubtaskExecutionAttemptAccumulatorsHeaders; import org.apache.flink.runtime.rest.messages.job.SubtaskExecutionAttemptAccumulatorsInfo; import org.apache.flink.runtime.rest.messages.job.UserAccumulator; -import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.OptionalFailure; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.concurrent.Executors; import org.junit.Test; @@ -68,7 +68,7 @@ public void testHandleRequest() throws Exception { new DefaultExecutionGraphCache( restHandlerConfiguration.getTimeout(), Time.milliseconds(restHandlerConfiguration.getRefreshInterval())), - TestingUtils.defaultExecutor()); + Executors.directExecutor()); // Instance a empty request. final HandlerRequest request = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptDetailsHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptDetailsHandlerTest.java index 57bb24439898e..5e19277f5cd37 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptDetailsHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/SubtaskExecutionAttemptDetailsHandlerTest.java @@ -46,8 +46,8 @@ import org.apache.flink.runtime.rest.messages.job.SubtaskExecutionAttemptDetailsInfo; import org.apache.flink.runtime.rest.messages.job.metrics.IOMetricsInfo; import org.apache.flink.runtime.util.EvictingBoundedList; -import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.concurrent.Executors; import org.junit.Test; @@ -112,7 +112,7 @@ public void testHandleRequest() throws Exception { new MetricFetcherImpl<>( () -> null, address -> null, - TestingUtils.defaultExecutor(), + Executors.directExecutor(), Time.milliseconds(1000L), MetricOptions.METRIC_FETCHER_UPDATE_INTERVAL.defaultValue()); @@ -129,7 +129,7 @@ public void testHandleRequest() throws Exception { new DefaultExecutionGraphCache( restHandlerConfiguration.getTimeout(), Time.milliseconds(restHandlerConfiguration.getRefreshInterval())), - TestingUtils.defaultExecutor(), + Executors.directExecutor(), metricFetcher); final HashMap receivedPathParameters = new HashMap<>(4); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AggregatingMetricsHandlerTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AggregatingMetricsHandlerTestBase.java index 0369e62fbc1af..c96a541796db1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AggregatingMetricsHandlerTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/metrics/AggregatingMetricsHandlerTestBase.java @@ -67,7 +67,6 @@ public abstract class AggregatingMetricsHandlerTestBase< private static final GatewayRetriever LEADER_RETRIEVER; private static final Time TIMEOUT = Time.milliseconds(50); private static final Map TEST_HEADERS = Collections.emptyMap(); - private static final Executor EXECUTOR = TestingUtils.defaultExecutor(); static { TEST_REST_ADDRESS = CompletableFuture.completedFuture("localhost:12345"); @@ -103,7 +102,13 @@ public void setUp() throws Exception { store.add(dump); } - handler = getHandler(LEADER_RETRIEVER, TIMEOUT, TEST_HEADERS, EXECUTOR, fetcher); + handler = + getHandler( + LEADER_RETRIEVER, + TIMEOUT, + TEST_HEADERS, + Executors.directExecutor(), + fetcher); pathParameters = getPathParameters(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionGraphFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionGraphFactoryTest.java index 42e2a35948243..9b31a0a0d1ea8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionGraphFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionGraphFactoryTest.java @@ -39,6 +39,7 @@ import org.apache.flink.runtime.shuffle.ShuffleTestUtils; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.hamcrest.MatcherAssert; @@ -51,6 +52,7 @@ import java.io.File; import java.io.IOException; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.Matchers.notNullValue; import static org.junit.Assert.assertThat; @@ -61,6 +63,10 @@ public class DefaultExecutionGraphFactoryTest extends TestLogger { @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Test public void testRestoringModifiedJobFromSavepointFails() throws Exception { final JobGraph jobGraphWithNewOperator = createJobGraphWithSavepoint(false, 42L); @@ -123,8 +129,8 @@ private ExecutionGraphFactory createExecutionGraphFactory() { new Configuration(), ClassLoader.getSystemClassLoader(), new DefaultExecutionDeploymentTracker(), - TestingUtils.defaultExecutor(), - TestingUtils.defaultExecutor(), + EXECUTOR_RESOURCE.getExecutor(), + EXECUTOR_RESOURCE.getExecutor(), Time.milliseconds(0L), UnregisteredMetricGroups.createUnregisteredJobManagerJobMetricGroup(), VoidBlobWriter.getInstance(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultOperatorCoordinatorHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultOperatorCoordinatorHandlerTest.java index d1f1282706081..f990b8f4c6e79 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultOperatorCoordinatorHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultOperatorCoordinatorHandlerTest.java @@ -29,11 +29,15 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.TestingOperatorCoordinator; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.SerializedValue; +import org.junit.ClassRule; import org.junit.Test; import java.io.IOException; +import java.util.concurrent.ScheduledExecutorService; import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createNoOpVertex; import static org.apache.flink.runtime.io.network.partition.ResultPartitionType.BLOCKING; @@ -43,6 +47,9 @@ /** Test for {@link DefaultOperatorCoordinatorHandler}. */ public class DefaultOperatorCoordinatorHandlerTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); @Test public void testRegisterAndStartNewCoordinators() throws Exception { @@ -94,6 +101,6 @@ private JobVertex[] createJobVertices(ResultPartitionType resultPartitionType) private DefaultExecutionGraph createDynamicGraph(JobVertex... jobVertices) throws Exception { return TestingDefaultExecutionGraphBuilder.newBuilder() .setJobGraph(new JobGraph(new JobID(), "TestJob", jobVertices)) - .buildDynamicGraph(); + .buildDynamicGraph(EXECUTOR_RESOURCE.getExecutor()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerBatchSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerBatchSchedulingTest.java index 1f154f6701251..2b72068abc76e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerBatchSchedulingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerBatchSchedulingTest.java @@ -42,11 +42,14 @@ import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.util.function.CheckedSupplier; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,6 +69,10 @@ public class DefaultSchedulerBatchSchedulingTest extends TestLogger { protected final Logger log = LoggerFactory.getLogger(getClass()); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static ScheduledExecutorService singleThreadScheduledExecutorService; private static ComponentMainThreadExecutor mainThreadExecutor; @@ -198,7 +205,8 @@ private SchedulerNG createScheduler( Time slotRequestTimeout, JobStatusListener jobStatusListener) throws Exception { - return SchedulerTestingUtils.newSchedulerBuilder(jobGraph, mainThreadExecutor) + return new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, mainThreadExecutor, EXECUTOR_RESOURCE.getExecutor()) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( physicalSlotProvider, slotRequestTimeout)) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java index 387d6679df685..fe70810021459 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java @@ -1598,10 +1598,11 @@ public void testCheckpointCleanerIsClosedAfterCheckpointServices() throws Except final JobGraph jobGraph = singleJobVertexJobGraph(1); enableCheckpointing(jobGraph); try { - return SchedulerTestingUtils.newSchedulerBuilder( + return new SchedulerTestingUtils.DefaultSchedulerBuilder( jobGraph, ComponentMainThreadExecutorServiceAdapter - .forSingleThreadExecutor(executorService)) + .forSingleThreadExecutor(executorService), + executorService) .setCheckpointRecoveryFactory(checkpointRecoveryFactory) .setCheckpointCleaner(checkpointCleaner) .build(); @@ -1845,12 +1846,14 @@ private DefaultScheduler createScheduler( private SchedulerTestingUtils.DefaultSchedulerBuilder createSchedulerBuilder( final JobGraph jobGraph, final ComponentMainThreadExecutor mainThreadExecutor) throws Exception { - return SchedulerTestingUtils.newSchedulerBuilder(jobGraph, mainThreadExecutor) + return new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, + mainThreadExecutor, + executor, + scheduledExecutorService, + taskRestartExecutor) .setLogger(log) - .setIoExecutor(executor) .setJobMasterConfiguration(configuration) - .setFutureExecutor(scheduledExecutorService) - .setDelayExecutor(taskRestartExecutor) .setSchedulingStrategyFactory(new PipelinedRegionSchedulingStrategy.Factory()) .setFailoverStrategyFactory(new RestartPipelinedRegionFailoverStrategy.Factory()) .setRestartBackoffTimeStrategy(testRestartBackoffTimeStrategy) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphToInputsLocationsRetrieverAdapterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphToInputsLocationsRetrieverAdapterTest.java index b522e9e90cd9a..583b77e099c6e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphToInputsLocationsRetrieverAdapterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphToInputsLocationsRetrieverAdapterTest.java @@ -30,14 +30,18 @@ import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; import org.junit.Test; import java.util.Collection; import java.util.Collections; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.hasItem; @@ -51,6 +55,10 @@ /** Tests for {@link ExecutionGraphToInputsLocationsRetrieverAdapter}. */ public class ExecutionGraphToInputsLocationsRetrieverAdapterTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + /** Tests that can get the producers of consumed result partitions. */ @Test public void testGetConsumedResultPartitionsProducers() throws Exception { @@ -63,7 +71,8 @@ public void testGetConsumedResultPartitionsProducers() throws Exception { producer2, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED); final ExecutionGraph eg = - ExecutionGraphTestUtils.createSimpleTestGraph(producer1, producer2, consumer); + ExecutionGraphTestUtils.createExecutionGraph( + EXECUTOR_RESOURCE.getExecutor(), producer1, producer2, consumer); final ExecutionGraphToInputsLocationsRetrieverAdapter inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(eg); @@ -90,7 +99,9 @@ public void testGetConsumedResultPartitionsProducers() throws Exception { public void testGetEmptyTaskManagerLocationIfVertexNotScheduled() throws Exception { final JobVertex jobVertex = ExecutionGraphTestUtils.createNoOpVertex(1); - final ExecutionGraph eg = ExecutionGraphTestUtils.createSimpleTestGraph(jobVertex); + final ExecutionGraph eg = + ExecutionGraphTestUtils.createExecutionGraph( + EXECUTOR_RESOURCE.getExecutor(), jobVertex); final ExecutionGraphToInputsLocationsRetrieverAdapter inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(eg); @@ -108,7 +119,9 @@ public void testGetTaskManagerLocationWhenScheduled() throws Exception { final TestingLogicalSlot testingLogicalSlot = new TestingLogicalSlotBuilder().createTestingLogicalSlot(); - final ExecutionGraph eg = ExecutionGraphTestUtils.createSimpleTestGraph(jobVertex); + final ExecutionGraph eg = + ExecutionGraphTestUtils.createExecutionGraph( + EXECUTOR_RESOURCE.getExecutor(), jobVertex); final ExecutionGraphToInputsLocationsRetrieverAdapter inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(eg); @@ -136,7 +149,9 @@ public void testGetTaskManagerLocationWhenScheduled() throws Exception { public void testGetNonExistingExecutionVertexWillThrowException() throws Exception { final JobVertex jobVertex = ExecutionGraphTestUtils.createNoOpVertex(1); - final ExecutionGraph eg = ExecutionGraphTestUtils.createSimpleTestGraph(jobVertex); + final ExecutionGraph eg = + ExecutionGraphTestUtils.createExecutionGraph( + EXECUTOR_RESOURCE.getExecutor(), jobVertex); final ExecutionGraphToInputsLocationsRetrieverAdapter inputsLocationsRetriever = new ExecutionGraphToInputsLocationsRetrieverAdapter(eg); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LocalInputPreferredSlotSharingStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LocalInputPreferredSlotSharingStrategyTest.java index 9cd9df0aa25c6..89e789a7604c6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LocalInputPreferredSlotSharingStrategyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/LocalInputPreferredSlotSharingStrategyTest.java @@ -35,9 +35,12 @@ import org.apache.flink.runtime.scheduler.strategy.TestingSchedulingExecutionVertex; import org.apache.flink.runtime.scheduler.strategy.TestingSchedulingTopology; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import java.util.Arrays; @@ -46,6 +49,7 @@ import java.util.List; import java.util.Objects; import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -56,6 +60,9 @@ /** Tests for {@link LocalInputPreferredSlotSharingStrategy}. */ public class LocalInputPreferredSlotSharingStrategyTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); private TestingSchedulingTopology topology; @@ -280,7 +287,9 @@ public void testInputLocalityIsRespectedWithTwoEdgesBetweenTwoVertices() throws final JobGraph jobGraph = JobGraphTestUtils.batchJobGraph(v1, v2); final ExecutionGraph executionGraph = - TestingDefaultExecutionGraphBuilder.newBuilder().setJobGraph(jobGraph).build(); + TestingDefaultExecutionGraphBuilder.newBuilder() + .setJobGraph(jobGraph) + .build(EXECUTOR_RESOURCE.getExecutor()); final SchedulingTopology topology = executionGraph.getSchedulingTopology(); final SlotSharingStrategy strategy = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerTestingUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerTestingUtils.java index aeb8bf0d6cc79..464d5c161db5b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerTestingUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerTestingUtils.java @@ -67,7 +67,6 @@ import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.taskexecutor.TaskExecutorOperatorEventGateway; import org.apache.flink.runtime.taskmanager.TaskExecutionState; -import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TernaryBoolean; import org.apache.flink.util.concurrent.ScheduledExecutor; @@ -102,43 +101,49 @@ public class SchedulerTestingUtils { private SchedulerTestingUtils() {} - public static DefaultSchedulerBuilder newSchedulerBuilder( - final JobGraph jobGraph, final ComponentMainThreadExecutor mainThreadExecutor) { - return new DefaultSchedulerBuilder(jobGraph, mainThreadExecutor); - } - public static DefaultScheduler createScheduler( - final JobGraph jobGraph, final ComponentMainThreadExecutor mainThreadExecutor) + final JobGraph jobGraph, + final ComponentMainThreadExecutor mainThreadExecutor, + final ScheduledExecutorService executorService) throws Exception { - return newSchedulerBuilder(jobGraph, mainThreadExecutor).build(); + return new DefaultSchedulerBuilder(jobGraph, mainThreadExecutor, executorService).build(); } public static DefaultSchedulerBuilder createSchedulerBuilder( - JobGraph jobGraph, ComponentMainThreadExecutor mainThreadExecutor) { + JobGraph jobGraph, + ComponentMainThreadExecutor mainThreadExecutor, + ScheduledExecutorService scheduledExecutorService) { return createSchedulerBuilder( - jobGraph, mainThreadExecutor, new SimpleAckingTaskManagerGateway()); + jobGraph, + mainThreadExecutor, + new SimpleAckingTaskManagerGateway(), + scheduledExecutorService); } public static DefaultSchedulerBuilder createSchedulerBuilder( JobGraph jobGraph, ComponentMainThreadExecutor mainThreadExecutor, - TaskExecutorOperatorEventGateway operatorEventGateway) { + TaskExecutorOperatorEventGateway operatorEventGateway, + ScheduledExecutorService scheduledExecutorService) { final TaskManagerGateway gateway = operatorEventGateway instanceof TaskManagerGateway ? (TaskManagerGateway) operatorEventGateway : new TaskExecutorOperatorEventGatewayAdapter(operatorEventGateway); - return createSchedulerBuilder(jobGraph, mainThreadExecutor, gateway); + return createSchedulerBuilder( + jobGraph, mainThreadExecutor, gateway, scheduledExecutorService); } - public static DefaultSchedulerBuilder createSchedulerBuilder( + private static DefaultSchedulerBuilder createSchedulerBuilder( JobGraph jobGraph, ComponentMainThreadExecutor mainThreadExecutor, - TaskManagerGateway taskManagerGateway) { + TaskManagerGateway taskManagerGateway, + ScheduledExecutorService executorService) { - return newSchedulerBuilder(jobGraph, mainThreadExecutor) + return new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, mainThreadExecutor, executorService) .setSchedulingStrategyFactory(new PipelinedRegionSchedulingStrategy.Factory()) .setRestartBackoffTimeStrategy(new TestRestartBackoffTimeStrategy(true, 0)) .setExecutionSlotAllocatorFactory( @@ -395,11 +400,10 @@ public static class DefaultSchedulerBuilder { new PipelinedRegionSchedulingStrategy.Factory(); protected Logger log = LOG; - protected Executor ioExecutor = TestingUtils.defaultExecutor(); + protected Executor ioExecutor; protected Configuration jobMasterConfiguration = new Configuration(); - protected ScheduledExecutorService futureExecutor = TestingUtils.defaultExecutor(); - protected ScheduledExecutor delayExecutor = - new ScheduledExecutorServiceAdapter(futureExecutor); + protected ScheduledExecutorService futureExecutor; + protected ScheduledExecutor delayExecutor; protected ClassLoader userCodeLoader = ClassLoader.getSystemClassLoader(); protected CheckpointsCleaner checkpointCleaner = new CheckpointsCleaner(); protected CheckpointRecoveryFactory checkpointRecoveryFactory = @@ -424,9 +428,28 @@ public static class DefaultSchedulerBuilder { protected JobStatusListener jobStatusListener = (ignoredA, ignoredB, ignoredC) -> {}; public DefaultSchedulerBuilder( - final JobGraph jobGraph, ComponentMainThreadExecutor mainThreadExecutor) { + final JobGraph jobGraph, + ComponentMainThreadExecutor mainThreadExecutor, + ScheduledExecutorService generalExecutorService) { + this( + jobGraph, + mainThreadExecutor, + generalExecutorService, + generalExecutorService, + new ScheduledExecutorServiceAdapter(generalExecutorService)); + } + + public DefaultSchedulerBuilder( + final JobGraph jobGraph, + ComponentMainThreadExecutor mainThreadExecutor, + Executor ioExecutor, + ScheduledExecutorService futureExecutor, + ScheduledExecutor delayExecuto) { this.jobGraph = jobGraph; this.mainThreadExecutor = mainThreadExecutor; + this.ioExecutor = ioExecutor; + this.futureExecutor = futureExecutor; + this.delayExecutor = delayExecuto; } public DefaultSchedulerBuilder setLogger(final Logger log) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SsgNetworkMemoryCalculationUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SsgNetworkMemoryCalculationUtilsTest.java index 2c1bf6421d54e..817d7f3dc41e0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SsgNetworkMemoryCalculationUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SsgNetworkMemoryCalculationUtilsTest.java @@ -41,7 +41,10 @@ import org.apache.flink.runtime.shuffle.ShuffleMaster; import org.apache.flink.runtime.shuffle.TaskInputsOutputsDescriptor; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; +import org.junit.ClassRule; import org.junit.Test; import java.util.Arrays; @@ -49,6 +52,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; @@ -56,6 +60,9 @@ /** Tests for {@link SsgNetworkMemoryCalculationUtils}. */ public class SsgNetworkMemoryCalculationUtilsTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); private static final TestShuffleMaster SHUFFLE_MASTER = new TestShuffleMaster(); @@ -180,7 +187,8 @@ private void testGetMaxInputChannelNumForResult( -1, consumerMaxParallelism, distributionPattern, - true); + true, + EXECUTOR_RESOURCE.getExecutor()); final Iterator vertexIterator = eg.getVerticesTopologically().iterator(); @@ -215,7 +223,7 @@ private DefaultExecutionGraph createDynamicExecutionGraph( .setJobGraph(jobGraph) .setVertexParallelismStore(vertexParallelismStore) .setShuffleMaster(SHUFFLE_MASTER) - .buildDynamicGraph(); + .buildDynamicGraph(EXECUTOR_RESOURCE.getExecutor()); } private void createExecutionGraphAndEnrichNetworkMemory( @@ -223,7 +231,7 @@ private void createExecutionGraphAndEnrichNetworkMemory( TestingDefaultExecutionGraphBuilder.newBuilder() .setJobGraph(createStreamingGraph(slotSharingGroups, Arrays.asList(4, 5, 6))) .setShuffleMaster(SHUFFLE_MASTER) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); } private static JobGraph createStreamingGraph( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopologyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopologyTest.java index 8cb20ad9c0880..109484fc1a1bc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopologyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultExecutionTopologyTest.java @@ -36,6 +36,8 @@ import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.scheduler.strategy.ResultPartitionState; import org.apache.flink.runtime.scheduler.strategy.SchedulingPipelinedRegion; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.IterableUtils; import org.apache.flink.util.TestLogger; @@ -43,6 +45,7 @@ import org.apache.flink.shaded.guava30.com.google.common.collect.Sets; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import java.util.ArrayList; @@ -53,11 +56,12 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; import static junit.framework.TestCase.assertSame; import static junit.framework.TestCase.assertTrue; +import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createExecutionGraph; import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createNoOpVertex; -import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.createSimpleTestGraph; import static org.apache.flink.runtime.io.network.partition.ResultPartitionType.BLOCKING; import static org.apache.flink.runtime.io.network.partition.ResultPartitionType.PIPELINED; import static org.apache.flink.runtime.jobgraph.DistributionPattern.ALL_TO_ALL; @@ -69,6 +73,9 @@ /** Unit tests for {@link DefaultExecutionTopology}. */ public class DefaultExecutionTopologyTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); private DefaultExecutionGraph executionGraph; @@ -81,7 +88,7 @@ public void setUp() throws Exception { jobVertices[0] = createNoOpVertex(parallelism); jobVertices[1] = createNoOpVertex(parallelism); jobVertices[1].connectNewDataSetAsInput(jobVertices[0], ALL_TO_ALL, PIPELINED); - executionGraph = createSimpleTestGraph(jobVertices); + executionGraph = createExecutionGraph(EXECUTOR_RESOURCE.getExecutor(), jobVertices); adapter = DefaultExecutionTopology.fromExecutionGraph(executionGraph); } @@ -169,7 +176,8 @@ public void testErrorIfCoLocatedTasksAreNotInSameRegion() throws Exception { v2.setSlotSharingGroup(slotSharingGroup); v1.setStrictlyCoLocatedWith(v2); - final DefaultExecutionGraph executionGraph = createSimpleTestGraph(v1, v2); + final DefaultExecutionGraph executionGraph = + createExecutionGraph(EXECUTOR_RESOURCE.getExecutor(), v1, v2); DefaultExecutionTopology.fromExecutionGraph(executionGraph); } @@ -245,7 +253,7 @@ private JobVertex[] createJobVertices(ResultPartitionType resultPartitionType) { private DefaultExecutionGraph createDynamicGraph(JobVertex... jobVertices) throws Exception { return TestingDefaultExecutionGraphBuilder.newBuilder() .setJobGraph(new JobGraph(new JobID(), "TestJob", jobVertices)) - .buildDynamicGraph(); + .buildDynamicGraph(EXECUTOR_RESOURCE.getExecutor()); } private void assertRegionContainsAllVertices( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingPipelinedRegionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingPipelinedRegionTest.java index 10cfebd3c8e95..949f039727c44 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingPipelinedRegionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adapter/DefaultSchedulingPipelinedRegionTest.java @@ -29,10 +29,13 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.scheduler.strategy.ConsumedPartitionGroup; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; +import org.junit.ClassRule; import org.junit.Test; import java.util.Collections; @@ -40,6 +43,7 @@ import java.util.Iterator; import java.util.Map; import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; @@ -51,6 +55,10 @@ /** Unit tests for {@link DefaultSchedulingPipelinedRegion}. */ public class DefaultSchedulingPipelinedRegionTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Test public void gettingUnknownVertexThrowsException() { final Map resultPartitionById = @@ -121,7 +129,8 @@ public void returnsIncidentBlockingPartitions() throws Exception { e.connectNewDataSetAsInput(d, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED); final DefaultExecutionGraph simpleTestGraph = - ExecutionGraphTestUtils.createSimpleTestGraph(a, b, c, d, e); + ExecutionGraphTestUtils.createExecutionGraph( + EXECUTOR_RESOURCE.getExecutor(), a, b, c, d, e); final DefaultExecutionTopology topology = DefaultExecutionTopology.fromExecutionGraph(simpleTestGraph); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerBuilder.java index 68a2e536996c1..8f3c65a63a1c3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerBuilder.java @@ -44,12 +44,10 @@ import org.apache.flink.runtime.scheduler.adaptive.allocator.SlotAllocator; import org.apache.flink.runtime.shuffle.ShuffleMaster; import org.apache.flink.runtime.shuffle.ShuffleTestUtils; -import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.FatalExitExceptionHandler; import javax.annotation.Nullable; -import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; /** Builder for {@link AdaptiveScheduler}. */ @@ -60,9 +58,7 @@ public class AdaptiveSchedulerBuilder { private final ComponentMainThreadExecutor mainThreadExecutor; - private Executor ioExecutor = TestingUtils.defaultExecutor(); private Configuration jobMasterConfiguration = new Configuration(); - private ScheduledExecutorService futureExecutor = TestingUtils.defaultExecutor(); private ClassLoader userCodeLoader = ClassLoader.getSystemClassLoader(); private CheckpointsCleaner checkpointsCleaner = new CheckpointsCleaner(); private CheckpointRecoveryFactory checkpointRecoveryFactory = @@ -99,23 +95,12 @@ public AdaptiveSchedulerBuilder( rpcTimeout); } - public AdaptiveSchedulerBuilder setIoExecutor(final Executor ioExecutor) { - this.ioExecutor = ioExecutor; - return this; - } - public AdaptiveSchedulerBuilder setJobMasterConfiguration( final Configuration jobMasterConfiguration) { this.jobMasterConfiguration = jobMasterConfiguration; return this; } - public AdaptiveSchedulerBuilder setFutureExecutor( - final ScheduledExecutorService futureExecutor) { - this.futureExecutor = futureExecutor; - return this; - } - public AdaptiveSchedulerBuilder setUserCodeLoader(final ClassLoader userCodeLoader) { this.userCodeLoader = userCodeLoader; return this; @@ -192,14 +177,14 @@ public AdaptiveSchedulerBuilder setSlotAllocator(SlotAllocator slotAllocator) { return this; } - public AdaptiveScheduler build() throws Exception { + public AdaptiveScheduler build(ScheduledExecutorService executorService) throws Exception { final ExecutionGraphFactory executionGraphFactory = new DefaultExecutionGraphFactory( jobMasterConfiguration, userCodeLoader, new DefaultExecutionDeploymentTracker(), - futureExecutor, - ioExecutor, + executorService, + executorService, rpcTimeout, jobManagerJobMetricGroup, blobWriter, @@ -214,7 +199,7 @@ public AdaptiveScheduler build() throws Exception { ? AdaptiveSchedulerFactory.createSlotSharingSlotAllocator( declarativeSlotPool) : slotAllocator, - ioExecutor, + executorService, userCodeLoader, checkpointsCleaner, checkpointRecoveryFactory, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java index 6e6f6d3d60036..c1bd404f118bf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java @@ -89,6 +89,7 @@ import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.util.ResourceCounter; import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.testutils.TestingUtils; import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.FlinkException; import org.apache.flink.util.IterableUtils; @@ -134,6 +135,10 @@ public class AdaptiveSchedulerTest extends TestLogger { private static final int PARALLELISM = 4; private static final JobVertex JOB_VERTEX = createNoOpVertex("v1", PARALLELISM); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @ClassRule @@ -152,7 +157,8 @@ public class AdaptiveSchedulerTest extends TestLogger { @Test public void testInitialState() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); assertThat(scheduler.getState()).isInstanceOf(Created.class); } @@ -167,7 +173,7 @@ public void testArchivedCheckpointingSettingsNotNullIfCheckpointingIsEnabled() final ArchivedExecutionGraph archivedExecutionGraph = new AdaptiveSchedulerBuilder(jobGraph, mainThreadExecutor) - .build() + .build(EXECUTOR_RESOURCE.getExecutor()) .getArchivedExecutionGraph(JobStatus.INITIALIZING, null); ArchivedExecutionGraphTest.assertContainsCheckpointSettings(archivedExecutionGraph); @@ -176,7 +182,8 @@ public void testArchivedCheckpointingSettingsNotNullIfCheckpointingIsEnabled() @Test public void testIsState() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); final State state = scheduler.getState(); @@ -187,7 +194,8 @@ public void testIsState() throws Exception { @Test public void testRunIfState() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); AtomicBoolean ran = new AtomicBoolean(false); scheduler.runIfState(scheduler.getState(), () -> ran.set(true)); @@ -197,7 +205,8 @@ public void testRunIfState() throws Exception { @Test public void testRunIfStateWithStateMismatch() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); AtomicBoolean ran = new AtomicBoolean(false); scheduler.runIfState(new DummyState(), () -> ran.set(true)); @@ -207,7 +216,8 @@ public void testRunIfStateWithStateMismatch() throws Exception { @Test public void testHasEnoughResourcesReturnsFalseIfUnsatisfied() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); scheduler.startScheduling(); @@ -227,7 +237,7 @@ public void testHasEnoughResourcesReturnsTrueIfSatisfied() throws Exception { final AdaptiveScheduler scheduler = new AdaptiveSchedulerBuilder(jobGraph, mainThreadExecutor) .setDeclarativeSlotPool(declarativeSlotPool) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); scheduler.startScheduling(); @@ -250,7 +260,7 @@ public void testHasEnoughResourcesUsesUnmatchedSlotsAsUnknown() throws Exception final AdaptiveScheduler scheduler = new AdaptiveSchedulerBuilder(jobGraph, mainThreadExecutor) .setDeclarativeSlotPool(declarativeSlotPool) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); scheduler.startScheduling(); @@ -280,7 +290,7 @@ public void testExecutionGraphGenerationWithAvailableResources() throws Exceptio new AdaptiveSchedulerBuilder(jobGraph, singleThreadMainThreadExecutor) .setDeclarativeSlotPool(declarativeSlotPool) .setJobMasterConfiguration(configuration) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final int numAvailableSlots = 1; @@ -327,7 +337,7 @@ public void testExecutionGraphGenerationSetsInitializationTimestamp() throws Exc .setInitializationTimestamp(initializationTimestamp) .setDeclarativeSlotPool(declarativeSlotPool) .setJobMasterConfiguration(configuration) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final SubmissionBufferingTaskManagerGateway taskManagerGateway = new SubmissionBufferingTaskManagerGateway(PARALLELISM); @@ -363,7 +373,7 @@ public void testInitializationTimestampForwarding() throws Exception { final AdaptiveScheduler adaptiveScheduler = new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) .setInitializationTimestamp(expectedInitializationTimestamp) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final long initializationTimestamp = adaptiveScheduler @@ -381,7 +391,7 @@ public void testFatalErrorsForwardedToFatalErrorHandler() throws Exception { final AdaptiveScheduler scheduler = new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) .setFatalErrorHandler(fatalErrorHandler) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final RuntimeException exception = new RuntimeException(); @@ -406,7 +416,7 @@ public void testResourceTimeout() throws Exception { final AdaptiveScheduler scheduler = new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) .setJobMasterConfiguration(configuration) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); scheduler.startScheduling(); @@ -459,7 +469,7 @@ public void testNumRestartsMetric() throws Exception { metricRegistry, "localhost") .addJob(new JobID(), "jobName")) .setDeclarativeSlotPool(declarativeSlotPool) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final Gauge numRestartsMetric = numRestartsMetricFuture.get(); @@ -546,7 +556,7 @@ public void testStatusMetrics() throws Exception { metricRegistry, "localhost") .addJob(new JobID(), "jobName")) .setDeclarativeSlotPool(declarativeSlotPool) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final UpTimeGauge upTimeGauge = upTimeMetricFuture.get(); final DownTimeGauge downTimeGauge = downTimeMetricFuture.get(); @@ -607,7 +617,8 @@ public void testStatusMetrics() throws Exception { @Test public void testStartSchedulingTransitionsToWaitingForResources() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); scheduler.startScheduling(); @@ -624,7 +635,7 @@ public void testStartSchedulingSetsResourceRequirementsForDefaultMode() throws E final AdaptiveScheduler scheduler = new AdaptiveSchedulerBuilder(jobGraph, mainThreadExecutor) .setDeclarativeSlotPool(declarativeSlotPool) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); scheduler.startScheduling(); @@ -646,7 +657,7 @@ public void testStartSchedulingSetsResourceRequirementsForReactiveMode() throws new AdaptiveSchedulerBuilder(jobGraph, mainThreadExecutor) .setDeclarativeSlotPool(declarativeSlotPool) .setJobMasterConfiguration(configuration) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); scheduler.startScheduling(); @@ -672,7 +683,7 @@ public void testResourceAcquisitionTriggersJobExecution() throws Exception { new AdaptiveSchedulerBuilder(jobGraph, singleThreadMainThreadExecutor) .setDeclarativeSlotPool(declarativeSlotPool) .setJobMasterConfiguration(configuration) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final SubmissionBufferingTaskManagerGateway taskManagerGateway = new SubmissionBufferingTaskManagerGateway(PARALLELISM); @@ -708,7 +719,8 @@ public void testResourceAcquisitionTriggersJobExecution() throws Exception { @Test public void testGoToFinished() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); final ArchivedExecutionGraph archivedExecutionGraph = new ArchivedExecutionGraphBuilder().setState(JobStatus.FAILED).build(); @@ -726,7 +738,7 @@ public void testJobStatusListenerOnlyCalledIfJobStatusChanges() throws Exception .setJobStatusListener( (jobId, newJobStatus, timestamp) -> numStatusUpdates.incrementAndGet()) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); // sanity check assertThat(scheduler.requestJobStatus()) @@ -773,7 +785,7 @@ public void testJobStatusListenerNotifiedOfJobStatusChanges() throws Exception { } }) .setDeclarativeSlotPool(declarativeSlotPool) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final SubmissionBufferingTaskManagerGateway taskManagerGateway = new SubmissionBufferingTaskManagerGateway(1 + PARALLELISM); @@ -832,7 +844,7 @@ public void testCloseShutsDownCheckpointingComponents() throws Exception { .setCheckpointRecoveryFactory( new TestingCheckpointRecoveryFactory( completedCheckpointStore, checkpointIdCounter)) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); singleThreadMainThreadExecutor.execute( () -> { @@ -849,7 +861,8 @@ public void testCloseShutsDownCheckpointingComponents() throws Exception { @Test public void testTransitionToStateCallsOnLeave() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); final LifecycleMethodCapturingState firstState = new LifecycleMethodCapturingState(); @@ -880,7 +893,7 @@ public void testConsistentMaxParallelism() throws Exception { new AdaptiveSchedulerBuilder(jobGraph, singleThreadMainThreadExecutor) .setDeclarativeSlotPool(declarativeSlotPool) .setJobMasterConfiguration(configuration) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final SubmissionBufferingTaskManagerGateway taskManagerGateway = new SubmissionBufferingTaskManagerGateway(1 + parallelism); @@ -943,7 +956,7 @@ public void testHowToHandleFailureRejectedByStrategy() throws Exception { final AdaptiveScheduler scheduler = new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) .setRestartBackoffTimeStrategy(NoRestartBackoffTimeStrategy.INSTANCE) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); assertThat(scheduler.howToHandleFailure(new Exception("test")).canRestart()).isFalse(); } @@ -956,7 +969,7 @@ public void testHowToHandleFailureAllowedByStrategy() throws Exception { final AdaptiveScheduler scheduler = new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) .setRestartBackoffTimeStrategy(restartBackoffTimeStrategy) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final FailureResult failureResult = scheduler.howToHandleFailure(new Exception("test")); @@ -968,7 +981,8 @@ public void testHowToHandleFailureAllowedByStrategy() throws Exception { @Test public void testHowToHandleFailureUnrecoverableFailure() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); assertThat( scheduler @@ -1052,7 +1066,7 @@ private Iterable runExceptionHistoryTests( .setCheckpointCleaner(checkpointCleaner) .setJobStatusListener(listener); setupScheduler.accept(builder); - final AdaptiveScheduler scheduler = builder.build(); + final AdaptiveScheduler scheduler = builder.build(EXECUTOR_RESOURCE.getExecutor()); final SubmissionBufferingTaskManagerGateway taskManagerGateway = new SubmissionBufferingTaskManagerGateway(numAvailableSlots); @@ -1288,7 +1302,8 @@ public void testExceptionHistoryWithTaskConcurrentFailure() throws Exception { @Test(expected = IllegalStateException.class) public void testRepeatedTransitionIntoCurrentStateFails() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); final State state = scheduler.getState(); @@ -1305,7 +1320,8 @@ public void testRepeatedTransitionIntoCurrentStateFails() throws Exception { @Test public void testTriggerSavepointFailsInIllegalState() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); assertThat( scheduler.triggerSavepoint( @@ -1318,7 +1334,8 @@ public void testTriggerSavepointFailsInIllegalState() throws Exception { @Test public void testStopWithSavepointFailsInIllegalState() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); assertThat( scheduler.triggerSavepoint( @@ -1331,7 +1348,8 @@ public void testStopWithSavepointFailsInIllegalState() throws Exception { @Test(expected = TaskNotRunningException.class) public void testDeliverOperatorEventToCoordinatorFailsInIllegalState() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); scheduler.deliverOperatorEventToCoordinator( new ExecutionAttemptID(), new OperatorID(), new TestOperatorEvent()); @@ -1340,7 +1358,8 @@ public void testDeliverOperatorEventToCoordinatorFailsInIllegalState() throws Ex @Test public void testDeliverCoordinationRequestToCoordinatorFailsInIllegalState() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); assertThat( scheduler.deliverCoordinationRequestToCoordinator( @@ -1354,7 +1373,8 @@ public void testDeliverCoordinationRequestToCoordinatorFailsInIllegalState() thr public void testUpdateTaskExecutionStateReturnsFalseInIllegalState() throws Exception { final JobGraph jobGraph = createJobGraph(); final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(jobGraph, mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(jobGraph, mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); assertThat( scheduler.updateTaskExecutionState( @@ -1367,7 +1387,8 @@ public void testUpdateTaskExecutionStateReturnsFalseInIllegalState() throws Exce @Test(expected = IOException.class) public void testRequestNextInputSplitFailsInIllegalState() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); scheduler.requestNextInputSplit(JOB_VERTEX.getID(), new ExecutionAttemptID()); } @@ -1375,7 +1396,8 @@ public void testRequestNextInputSplitFailsInIllegalState() throws Exception { @Test(expected = PartitionProducerDisposedException.class) public void testRequestPartitionStateFailsInIllegalState() throws Exception { final AdaptiveScheduler scheduler = - new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor).build(); + new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) + .build(EXECUTOR_RESOURCE.getExecutor()); scheduler.requestPartitionState(new IntermediateDataSetID(), new ResultPartitionID()); } @@ -1392,7 +1414,7 @@ public void testTryToAssignSlotsReturnsNotPossibleIfExpectedResourcesAreNotAvail final AdaptiveScheduler adaptiveScheduler = new AdaptiveSchedulerBuilder(createJobGraph(), mainThreadExecutor) .setSlotAllocator(slotAllocator) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final CreatingExecutionGraph.AssignmentResult assignmentResult = adaptiveScheduler.tryToAssignSlots( @@ -1457,7 +1479,7 @@ public void testCheckpointCleanerIsClosedAfterCheckpointServices() throws Except .forSingleThreadExecutor(executorService)) .setCheckpointRecoveryFactory(checkpointRecoveryFactory) .setCheckpointCleaner(checkpointCleaner) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/ExecutingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/ExecutingTest.java index cd5c3af7499b6..bfd734dcd4dd6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/ExecutingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/ExecutingTest.java @@ -64,10 +64,13 @@ import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.shuffle.ShuffleMaster; import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.types.Either; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; import org.junit.Test; import org.slf4j.Logger; @@ -82,6 +85,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.function.Consumer; import java.util.function.Function; @@ -96,6 +100,9 @@ /** Tests for {@link AdaptiveScheduler AdaptiveScheduler's} {@link Executing} state. */ public class ExecutingTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); @Test public void testExecutionGraphDeploymentOnEnter() throws Exception { @@ -361,7 +368,8 @@ public void testExecutionVertexMarkedAsFailedOnDeploymentFailure() throws Except public void testTransitionToStopWithSavepointState() throws Exception { try (MockExecutingContext ctx = new MockExecutingContext()) { CheckpointCoordinator coordinator = - new CheckpointCoordinatorTestingUtils.CheckpointCoordinatorBuilder().build(); + new CheckpointCoordinatorTestingUtils.CheckpointCoordinatorBuilder() + .build(EXECUTOR_RESOURCE.getExecutor()); StateTrackingMockExecutionGraph mockedExecutionGraphWithCheckpointCoordinator = new StateTrackingMockExecutionGraph() { @Nullable @@ -384,7 +392,8 @@ public CheckpointCoordinator getCheckpointCoordinator() { public void testCheckpointSchedulerIsStoppedOnStopWithSavepoint() throws Exception { try (MockExecutingContext ctx = new MockExecutingContext()) { CheckpointCoordinator coordinator = - new CheckpointCoordinatorTestingUtils.CheckpointCoordinatorBuilder().build(); + new CheckpointCoordinatorTestingUtils.CheckpointCoordinatorBuilder() + .build(EXECUTOR_RESOURCE.getExecutor()); StateTrackingMockExecutionGraph mockedExecutionGraphWithCheckpointCoordinator = new StateTrackingMockExecutionGraph() { @Nullable @@ -474,7 +483,8 @@ public static TaskExecutionStateTransition createFailingStateTransition( private final class ExecutingStateBuilder { private ExecutionGraph executionGraph = - TestingDefaultExecutionGraphBuilder.newBuilder().build(); + TestingDefaultExecutionGraphBuilder.newBuilder() + .build(EXECUTOR_RESOURCE.getExecutor()); private OperatorCoordinatorHandler operatorCoordinatorHandler; private ExecutingStateBuilder() throws JobException, JobExecutionException { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchSchedulerTest.java index ce4aeb231448f..bf330d9edf3be 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchSchedulerTest.java @@ -38,13 +38,17 @@ import org.apache.flink.runtime.scheduler.SchedulerBase; import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; import org.junit.Test; import java.util.Arrays; import java.util.Iterator; import java.util.List; +import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; import static org.hamcrest.MatcherAssert.assertThat; @@ -56,6 +60,10 @@ public class AdaptiveBatchSchedulerTest extends TestLogger { private static final int SOURCE_PARALLELISM_1 = 6; private static final int SOURCE_PARALLELISM_2 = 4; + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static final ComponentMainThreadExecutor mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread(); @@ -172,7 +180,9 @@ public SchedulerBase createScheduler(JobGraph jobGraph) throws Exception { final AdaptiveBatchSchedulerTestUtils.AdaptiveBatchSchedulerBuilder schedulerBuilder = (AdaptiveBatchSchedulerTestUtils.AdaptiveBatchSchedulerBuilder) new AdaptiveBatchSchedulerTestUtils.AdaptiveBatchSchedulerBuilder( - jobGraph, mainThreadExecutor) + jobGraph, + mainThreadExecutor, + EXECUTOR_RESOURCE.getExecutor()) .setJobMasterConfiguration(configuration); schedulerBuilder.setVertexParallelismDecider((ignored) -> 10); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchSchedulerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchSchedulerTestUtils.java index 84240214088b0..98152ee95462d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchSchedulerTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/AdaptiveBatchSchedulerTestUtils.java @@ -28,6 +28,8 @@ import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; import org.apache.flink.runtime.scheduler.strategy.VertexwiseSchedulingStrategy; +import java.util.concurrent.ScheduledExecutorService; + /** A utility class to create {@link AdaptiveBatchScheduler} instances for testing. */ public class AdaptiveBatchSchedulerTestUtils { @@ -41,8 +43,10 @@ public static class AdaptiveBatchSchedulerBuilder JobManagerOptions.ADAPTIVE_BATCH_SCHEDULER_MAX_PARALLELISM.defaultValue(); public AdaptiveBatchSchedulerBuilder( - JobGraph jobGraph, ComponentMainThreadExecutor mainThreadExecutor) { - super(jobGraph, mainThreadExecutor); + JobGraph jobGraph, + ComponentMainThreadExecutor mainThreadExecutor, + ScheduledExecutorService executorService) { + super(jobGraph, mainThreadExecutor, executorService); setSchedulingStrategyFactory(new VertexwiseSchedulingStrategy.Factory()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/forwardgroup/ForwardGroupComputeUtilTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/forwardgroup/ForwardGroupComputeUtilTest.java index b0d19f1b219b8..dbf6cb990622b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/forwardgroup/ForwardGroupComputeUtilTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptivebatch/forwardgroup/ForwardGroupComputeUtilTest.java @@ -28,13 +28,17 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.scheduler.adaptivebatch.AdaptiveBatchScheduler; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; import org.junit.Test; import java.util.Arrays; import java.util.HashSet; import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -42,6 +46,10 @@ /** Unit tests for {@link ForwardGroupComputeUtil}. */ public class ForwardGroupComputeUtilTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + /** * Tests that the computation of the job graph with isolated vertices works correctly. * @@ -200,6 +208,6 @@ private static DefaultExecutionGraph createDynamicGraph(JobVertex... vertices) .setVertexParallelismStore( AdaptiveBatchScheduler.computeVertexParallelismStoreForDynamicGraph( Arrays.asList(vertices), 10)); - return builder.buildDynamicGraph(); + return builder.buildDynamicGraph(EXECUTOR_RESOURCE.getExecutor()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/SchedulerBenchmarkBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/SchedulerBenchmarkBase.java index bf50fe8793998..10250118d0d5b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/SchedulerBenchmarkBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/SchedulerBenchmarkBase.java @@ -18,8 +18,9 @@ package org.apache.flink.runtime.scheduler.benchmark; -import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.util.concurrent.ExecutorThreadFactory; +import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; /** Base class of all scheduler benchmarks. */ @@ -27,7 +28,9 @@ public class SchedulerBenchmarkBase { public ScheduledExecutorService scheduledExecutorService; public void setup() { - scheduledExecutorService = TestingUtils.defaultExecutor(); + scheduledExecutorService = + Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("flink-benchmarks")); } public void teardown() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/SchedulerBenchmarkUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/SchedulerBenchmarkUtils.java index 09e125093a9cb..1cffa39e075fa 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/SchedulerBenchmarkUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/SchedulerBenchmarkUtils.java @@ -104,7 +104,8 @@ public static ExecutionGraph createAndInitExecutionGraph( ComponentMainThreadExecutorServiceAdapter.forMainThread(); final DefaultScheduler scheduler = - SchedulerTestingUtils.createSchedulerBuilder(jobGraph, mainThreadExecutor) + SchedulerTestingUtils.createSchedulerBuilder( + jobGraph, mainThreadExecutor, scheduledExecutorService) .setIoExecutor(scheduledExecutorService) .setFutureExecutor(scheduledExecutorService) .setDelayExecutor( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/e2e/SchedulerEndToEndBenchmarkBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/e2e/SchedulerEndToEndBenchmarkBase.java index 0a05d8eb677c1..42fcf4cf317b9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/e2e/SchedulerEndToEndBenchmarkBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/e2e/SchedulerEndToEndBenchmarkBase.java @@ -31,7 +31,6 @@ import org.apache.flink.runtime.scheduler.SchedulerTestingUtils; import org.apache.flink.runtime.scheduler.benchmark.JobConfiguration; import org.apache.flink.runtime.scheduler.benchmark.SchedulerBenchmarkBase; -import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; import java.util.List; import java.util.concurrent.ScheduledExecutorService; @@ -77,10 +76,8 @@ static DefaultScheduler createScheduler( ComponentMainThreadExecutor mainThreadExecutor, ScheduledExecutorService executorService) throws Exception { - return SchedulerTestingUtils.newSchedulerBuilder(jobGraph, mainThreadExecutor) - .setFutureExecutor(executorService) - .setIoExecutor(executorService) - .setDelayExecutor(new ScheduledExecutorServiceAdapter(executorService)) + return new SchedulerTestingUtils.DefaultSchedulerBuilder( + jobGraph, mainThreadExecutor, executorService) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( physicalSlotProvider)) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/topology/BuildExecutionGraphBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/topology/BuildExecutionGraphBenchmark.java index 4618ac5f388c2..479febeffd412 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/topology/BuildExecutionGraphBenchmark.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/benchmark/topology/BuildExecutionGraphBenchmark.java @@ -54,9 +54,7 @@ public void setup(JobConfiguration jobConfiguration) throws Exception { TestingDefaultExecutionGraphBuilder.newBuilder() .setVertexParallelismStore(parallelismStore) .setJobGraph(jobGraph) - .setFutureExecutor(scheduledExecutorService) - .setIoExecutor(scheduledExecutorService) - .build(); + .build(scheduledExecutorService); } public void buildTopology() throws Exception { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/FailureHandlingResultSnapshotTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/FailureHandlingResultSnapshotTest.java index 3c8fa5939320e..1cb06e5cbafcf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/FailureHandlingResultSnapshotTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/FailureHandlingResultSnapshotTest.java @@ -35,6 +35,8 @@ import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.FlinkException; import org.apache.flink.util.SerializedThrowable; import org.apache.flink.util.TestLogger; @@ -45,9 +47,11 @@ import org.hamcrest.collection.IsIterableContainingInOrder; import org.hamcrest.core.IsInstanceOf; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import java.util.Collections; +import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -59,6 +63,9 @@ * FailureHandlingResultSnapshot}. */ public class FailureHandlingResultSnapshotTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); private ExecutionGraph executionGraph; @@ -68,7 +75,9 @@ public void setup() throws JobException, JobExecutionException { jobGraph.getVertices().forEach(v -> v.setParallelism(3)); executionGraph = - TestingDefaultExecutionGraphBuilder.newBuilder().setJobGraph(jobGraph).build(); + TestingDefaultExecutionGraphBuilder.newBuilder() + .setJobGraph(jobGraph) + .build(EXECUTOR_RESOURCE.getExecutor()); executionGraph.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/RootExceptionHistoryEntryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/RootExceptionHistoryEntryTest.java index 2b5beea1e7813..21954fdee0929 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/RootExceptionHistoryEntryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/exceptionhistory/RootExceptionHistoryEntryTest.java @@ -30,6 +30,8 @@ import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; import org.apache.flink.runtime.taskmanager.TaskExecutionState; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; @@ -37,9 +39,11 @@ import org.hamcrest.collection.IsIterableContainingInAnyOrder; import org.hamcrest.collection.IsIterableContainingInOrder; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import java.util.Collections; +import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -49,6 +53,9 @@ * {@code RootExceptionHistoryEntryTest} tests the creation of {@link RootExceptionHistoryEntry}. */ public class RootExceptionHistoryEntryTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); private ExecutionGraph executionGraph; @@ -58,7 +65,9 @@ public void setup() throws JobException, JobExecutionException { jobGraph.getVertices().forEach(v -> v.setParallelism(3)); executionGraph = - TestingDefaultExecutionGraphBuilder.newBuilder().setJobGraph(jobGraph).build(); + TestingDefaultExecutionGraphBuilder.newBuilder() + .setJobGraph(jobGraph) + .build(EXECUTOR_RESOURCE.getExecutor()); executionGraph.start(ComponentMainThreadExecutorServiceAdapter.forMainThread()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/PipelinedRegionSchedulingStrategyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/PipelinedRegionSchedulingStrategyTest.java index 421195561e336..670293e9b73d8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/PipelinedRegionSchedulingStrategyTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/strategy/PipelinedRegionSchedulingStrategyTest.java @@ -29,9 +29,12 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.scheduler.ExecutionVertexDeploymentOption; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import java.util.ArrayList; @@ -39,6 +42,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -50,6 +54,9 @@ /** Unit tests for {@link PipelinedRegionSchedulingStrategy}. */ public class PipelinedRegionSchedulingStrategyTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); private TestingSchedulerOperations testingSchedulerOperation; @@ -200,7 +207,9 @@ public void testComputingCrossRegionConsumedPartitionGroupsCorrectly() throws Ex final JobGraph jobGraph = JobGraphBuilder.newBatchJobGraphBuilder().addJobVertices(ordered).build(); final ExecutionGraph executionGraph = - TestingDefaultExecutionGraphBuilder.newBuilder().setJobGraph(jobGraph).build(); + TestingDefaultExecutionGraphBuilder.newBuilder() + .setJobGraph(jobGraph) + .build(EXECUTOR_RESOURCE.getExecutor()); final SchedulingTopology schedulingTopology = executionGraph.getSchedulingTopology(); @@ -262,7 +271,9 @@ public void testSchedulingTopologyWithCrossRegionConsumedPartitionGroups() throw final JobGraph jobGraph = JobGraphBuilder.newBatchJobGraphBuilder().addJobVertices(ordered).build(); final ExecutionGraph executionGraph = - TestingDefaultExecutionGraphBuilder.newBuilder().setJobGraph(jobGraph).build(); + TestingDefaultExecutionGraphBuilder.newBuilder() + .setJobGraph(jobGraph) + .build(EXECUTOR_RESOURCE.getExecutor()); final SchedulingTopology schedulingTopology = executionGraph.getSchedulingTopology(); @@ -328,7 +339,9 @@ public void testScheduleBlockingDownstreamTaskIndividually() throws Exception { final JobGraph jobGraph = JobGraphBuilder.newBatchJobGraphBuilder().addJobVertices(ordered).build(); final ExecutionGraph executionGraph = - TestingDefaultExecutionGraphBuilder.newBuilder().setJobGraph(jobGraph).build(); + TestingDefaultExecutionGraphBuilder.newBuilder() + .setJobGraph(jobGraph) + .build(EXECUTOR_RESOURCE.getExecutor()); final SchedulingTopology schedulingTopology = executionGraph.getSchedulingTopology(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorExecutionDeploymentReconciliationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorExecutionDeploymentReconciliationTest.java index b1c180868ce91..3ea88d398012b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorExecutionDeploymentReconciliationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorExecutionDeploymentReconciliationTest.java @@ -55,6 +55,8 @@ import org.apache.flink.runtime.taskexecutor.slot.TaskSlotUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource; import org.apache.flink.testutils.TestFileUtils; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; @@ -73,6 +75,7 @@ import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import java.util.stream.StreamSupport; import static org.hamcrest.collection.IsCollectionWithSize.hasSize; @@ -93,6 +96,10 @@ public class TaskExecutorExecutionDeploymentReconciliationTest extends TestLogge new SettableLeaderRetrievalService(); private final JobID jobId = new JobID(); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @ClassRule public static final TestingRpcServiceResource RPC_SERVICE_RESOURCE = new TestingRpcServiceResource(); @@ -132,7 +139,9 @@ public void testDeployedExecutionReporting() throws Exception { final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskSlotTable(TaskSlotUtils.createTaskSlotTable(1, timeout)) + .setTaskSlotTable( + TaskSlotUtils.createTaskSlotTable( + 1, timeout, EXECUTOR_RESOURCE.getExecutor())) .setShuffleEnvironment(new NettyShuffleEnvironmentBuilder().build()) .build(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorOperatorEventHandlingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorOperatorEventHandlingTest.java index 9014ec9aa7ef3..d44db746b616c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorOperatorEventHandlingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorOperatorEventHandlingTest.java @@ -39,12 +39,15 @@ import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.testutils.CancelableInvokable; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.FlinkException; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Test; import java.io.IOException; @@ -52,6 +55,7 @@ import java.util.Collections; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; import static org.apache.flink.core.testutils.FlinkMatchers.futureWillCompleteExceptionally; import static org.junit.Assert.assertEquals; @@ -60,6 +64,10 @@ /** Test for the (failure handling of the) delivery of Operator Events. */ public class TaskExecutorOperatorEventHandlingTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private MetricRegistryImpl metricRegistry; private TestingRpcService rpcService; @@ -169,7 +177,7 @@ private TaskSubmissionTestEnvironment createExecutorWithRunningTask( throw new RuntimeException(); }) .build()) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); env.getTaskSlotTable().allocateSlot(0, jobId, tdd.getAllocationId(), Time.seconds(60)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java index fa67ff022a262..6bf9aeba85618 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorPartitionLifecycleTest.java @@ -66,6 +66,7 @@ import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.testutils.TestFileUtils; +import org.apache.flink.testutils.TestingUtils; import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.Reference; import org.apache.flink.util.SerializedValue; @@ -91,6 +92,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.StreamSupport; @@ -119,8 +121,8 @@ public class TaskExecutorPartitionLifecycleTest extends TestLogger { @Rule public final TemporaryFolder tmp = new TemporaryFolder(); @ClassRule - public static final TestExecutorResource TEST_EXECUTOR_SERVICE_RESOURCE = - new TestExecutorResource<>(() -> java.util.concurrent.Executors.newFixedThreadPool(1)); + public static final TestExecutorResource + TEST_EXECUTOR_SERVICE_RESOURCE = TestingUtils.defaultExecutorResource(); @Before public void setup() { @@ -612,7 +614,8 @@ private TestingTaskExecutor createTestingTaskExecutor( } private static TaskSlotTable createTaskSlotTable() { - return TaskSlotUtils.createTaskSlotTable(1, timeout); + return TaskSlotUtils.createTaskSlotTable( + 1, timeout, TEST_EXECUTOR_SERVICE_RESOURCE.getExecutor()); } @FunctionalInterface diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSlotLifetimeTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSlotLifetimeTest.java index d6e591d39eddd..f91f9e72b37cb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSlotLifetimeTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSlotLifetimeTest.java @@ -49,6 +49,8 @@ import org.apache.flink.runtime.taskmanager.TaskExecutionState; import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource; import org.apache.flink.testutils.TestFileUtils; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.util.function.FunctionUtils; @@ -62,6 +64,7 @@ import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.CoreMatchers.sameInstance; import static org.junit.Assert.assertThat; @@ -69,6 +72,10 @@ /** Tests for the {@link TaskExecutor TaskExecutor's} slot lifetime and its dependencies. */ public class TaskExecutorSlotLifetimeTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @ClassRule public static final TestingRpcServiceResource TESTING_RPC_SERVICE_RESOURCE = new TestingRpcServiceResource(); @@ -210,7 +217,9 @@ private TaskExecutor createTaskExecutor( TestFileUtils.createTempDir()), haServices, new TaskManagerServicesBuilder() - .setTaskSlotTable(TaskSlotUtils.createTaskSlotTable(1)) + .setTaskSlotTable( + TaskSlotUtils.createTaskSlotTable( + 1, EXECUTOR_RESOURCE.getExecutor())) .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) .build(), ExternalResourceInfoProvider.NO_EXTERNAL_RESOURCES, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java index 63032fb025ea0..36443dca212cc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java @@ -58,6 +58,8 @@ import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.testtasks.BlockingNoOpInvokable; import org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.NetUtils; import org.apache.flink.util.Preconditions; @@ -65,6 +67,7 @@ import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; @@ -76,6 +79,7 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import static org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder.createRemoteWithIdAndLocation; import static org.hamcrest.CoreMatchers.is; @@ -89,6 +93,10 @@ /** Tests for submission logic of the {@link TaskExecutor}. */ public class TaskExecutorSubmissionTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Rule public final TestName testName = new TestName(); private static final Time timeout = Time.milliseconds(10000L); @@ -113,7 +121,7 @@ public void testTaskSubmission() throws Exception { .setSlotSize(1) .addTaskManagerActionListener( eid, ExecutionState.RUNNING, taskRunningFuture) - .build()) { + .build(EXECUTOR_RESOURCE.getExecutor())) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -141,7 +149,8 @@ public void testSubmitTaskFailure() throws Exception { // must be >= 1 try (TaskSubmissionTestEnvironment env = - new TaskSubmissionTestEnvironment.Builder(jobId).build()) { + new TaskSubmissionTestEnvironment.Builder(jobId) + .build(EXECUTOR_RESOURCE.getExecutor())) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -176,7 +185,7 @@ public void testTaskSubmissionAndCancelling() throws Exception { eid2, ExecutionState.RUNNING, task2RunningFuture) .addTaskManagerActionListener( eid1, ExecutionState.CANCELED, task1CanceledFuture) - .build()) { + .build(EXECUTOR_RESOURCE.getExecutor())) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -231,7 +240,7 @@ public void testGateChannelEdgeMismatch() throws Exception { .addTaskManagerActionListener( eid2, ExecutionState.FAILED, task2FailedFuture) .setSlotSize(2) - .build()) { + .build(EXECUTOR_RESOURCE.getExecutor())) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -292,7 +301,7 @@ public void testRunJobWithForwardChannel() throws Exception { .setJobMasterId(jobMasterId) .setJobMasterGateway(testingJobMasterGateway) .useRealNonMockShuffleEnvironment() - .build()) { + .build(EXECUTOR_RESOURCE.getExecutor())) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -369,7 +378,7 @@ public void testCancellingDependentAndStateUpdateFails() throws Exception { .setJobMasterId(jobMasterId) .setJobMasterGateway(testingJobMasterGateway) .useRealNonMockShuffleEnvironment() - .build()) { + .build(EXECUTOR_RESOURCE.getExecutor())) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -423,7 +432,7 @@ public void testRemotePartitionNotFound() throws Exception { .setConfiguration(config) .setLocalCommunication(false) .useRealNonMockShuffleEnvironment() - .build()) { + .build(EXECUTOR_RESOURCE.getExecutor())) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -459,7 +468,7 @@ public void testUpdateTaskInputPartitionsFailure() throws Exception { .addTaskManagerActionListener( eid, ExecutionState.RUNNING, taskRunningFuture) .addTaskManagerActionListener(eid, ExecutionState.FAILED, taskFailedFuture) - .build()) { + .build(EXECUTOR_RESOURCE.getExecutor())) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -517,7 +526,7 @@ public void testLocalPartitionNotFound() throws Exception { .addTaskManagerActionListener(eid, ExecutionState.FAILED, taskFailedFuture) .setConfiguration(config) .useRealNonMockShuffleEnvironment() - .build()) { + .build(EXECUTOR_RESOURCE.getExecutor())) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -582,7 +591,7 @@ public void testFailingNotifyPartitionDataAvailable() throws Exception { .setJobMasterId(jobMasterId) .setJobMasterGateway(testingJobMasterGateway) .useRealNonMockShuffleEnvironment() - .build()) { + .build(EXECUTOR_RESOURCE.getExecutor())) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); 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 a2852c2a9738e..1cd7e724f31fa 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 @@ -101,6 +101,8 @@ import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.testutils.TestFileUtils; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FlinkException; @@ -118,6 +120,7 @@ import org.junit.After; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -186,6 +189,10 @@ public class TaskExecutorTest extends TestLogger { MemorySize.parse("4m"), Collections.emptyList()); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Rule public final TemporaryFolder tmp = new TemporaryFolder(); @Rule public final TestName testName = new TestName(); @@ -255,7 +262,8 @@ public void teardown() throws Exception { @Test public void testShouldShutDownTaskManagerServicesInPostStop() throws Exception { - final TaskSlotTableImpl taskSlotTable = TaskSlotUtils.createTaskSlotTable(1); + final TaskSlotTableImpl taskSlotTable = + TaskSlotUtils.createTaskSlotTable(1, EXECUTOR_RESOURCE.getExecutor()); final JobLeaderService jobLeaderService = new DefaultJobLeaderService( @@ -382,7 +390,9 @@ private void runJobManagerHeartbeatTest( final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) - .setTaskSlotTable(TaskSlotUtils.createTaskSlotTable(1)) + .setTaskSlotTable( + TaskSlotUtils.createTaskSlotTable( + 1, EXECUTOR_RESOURCE.getExecutor())) .setJobLeaderService(jobLeaderService) .setTaskStateManager(localStateStoresManager) .build(); @@ -725,7 +735,9 @@ public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception { final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setUnresolvedTaskManagerLocation(unresolvedTaskManagerLocation) - .setTaskSlotTable(TaskSlotUtils.createTaskSlotTable(1)) + .setTaskSlotTable( + TaskSlotUtils.createTaskSlotTable( + 1, EXECUTOR_RESOURCE.getExecutor())) .setTaskStateManager(createTaskExecutorLocalStateStoresManager()) .build(); @@ -881,7 +893,8 @@ private ResourceManagerId createAndRegisterResourceManager( */ @Test public void testJobLeaderDetection() throws Exception { - final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(1); + final TaskSlotTable taskSlotTable = + TaskSlotUtils.createTaskSlotTable(1, EXECUTOR_RESOURCE.getExecutor()); final JobLeaderService jobLeaderService = new DefaultJobLeaderService( unresolvedTaskManagerLocation, @@ -992,7 +1005,8 @@ public void testSlotAcceptance() throws Exception { rpc.registerGateway(resourceManagerGateway.getAddress(), resourceManagerGateway); rpc.registerGateway(jobMasterGateway.getAddress(), jobMasterGateway); - final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(2); + final TaskSlotTable taskSlotTable = + TaskSlotUtils.createTaskSlotTable(2, EXECUTOR_RESOURCE.getExecutor()); final TaskManagerServices taskManagerServices = createTaskManagerServicesWithTaskSlotTable(taskSlotTable); final TestingTaskExecutor taskManager = createTestingTaskExecutor(taskManagerServices); @@ -1117,7 +1131,8 @@ private void testSlotOfferResponseWithPendingSlotOffer(final ResponseOrder respo rpc.registerGateway(resourceManagerGateway.getAddress(), resourceManagerGateway); rpc.registerGateway(jobMasterGateway.getAddress(), jobMasterGateway); - final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(2); + final TaskSlotTable taskSlotTable = + TaskSlotUtils.createTaskSlotTable(2, EXECUTOR_RESOURCE.getExecutor()); final TaskManagerServices taskManagerServices = createTaskManagerServicesWithTaskSlotTable(taskSlotTable); final TestingTaskExecutor taskExecutor = createTestingTaskExecutor(taskManagerServices); @@ -1237,7 +1252,8 @@ public void testSlotOfferCounterIsSeparatedByJob() throws Exception { rpc.registerGateway(jobMasterGateway1.getAddress(), jobMasterGateway1); rpc.registerGateway(jobMasterGateway2.getAddress(), jobMasterGateway2); - final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(2); + final TaskSlotTable taskSlotTable = + TaskSlotUtils.createTaskSlotTable(2, EXECUTOR_RESOURCE.getExecutor()); final TaskManagerServices taskManagerServices = createTaskManagerServicesWithTaskSlotTable(taskSlotTable); final TestingTaskExecutor taskExecutor = createTestingTaskExecutor(taskManagerServices); @@ -1331,7 +1347,8 @@ public void testFreeingInactiveSlotDoesNotFail() throws Exception { rpc.registerGateway(jobMasterGateway.getAddress(), jobMasterGateway); - final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(1); + final TaskSlotTable taskSlotTable = + TaskSlotUtils.createTaskSlotTable(1, EXECUTOR_RESOURCE.getExecutor()); final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); final TaskManagerServices taskManagerServices = @@ -1408,7 +1425,8 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { rpc.registerGateway(resourceManagerGateway.getAddress(), resourceManagerGateway); rpc.registerGateway(jobMasterGateway.getAddress(), jobMasterGateway); - final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(2); + final TaskSlotTable taskSlotTable = + TaskSlotUtils.createTaskSlotTable(2, EXECUTOR_RESOURCE.getExecutor()); final TaskManagerServices taskManagerServices = createTaskManagerServicesWithTaskSlotTable(taskSlotTable); final TestingTaskExecutor taskManager = createTestingTaskExecutor(taskManagerServices); @@ -1551,7 +1569,8 @@ public void testRMHeartbeatStopWhenLeadershipRevoked() throws Exception { new RecordingHeartbeatServices(heartbeatInterval, heartbeatTimeout); final ResourceID rmResourceID = ResourceID.generate(); - final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(1); + final TaskSlotTable taskSlotTable = + TaskSlotUtils.createTaskSlotTable(1, EXECUTOR_RESOURCE.getExecutor()); final String rmAddress = "rm"; final TestingResourceManagerGateway rmGateway = @@ -1609,7 +1628,8 @@ public void testRMHeartbeatStopWhenLeadershipRevoked() throws Exception { */ @Test public void testRemoveJobFromJobLeaderService() throws Exception { - final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(1); + final TaskSlotTable taskSlotTable = + TaskSlotUtils.createTaskSlotTable(1, EXECUTOR_RESOURCE.getExecutor()); final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); @@ -1715,7 +1735,8 @@ public void testMaximumRegistrationDuration() throws Exception { public void testMaximumRegistrationDurationAfterConnectionLoss() throws Exception { configuration.set( TaskManagerOptions.REGISTRATION_TIMEOUT, TimeUtils.parseDuration("100 ms")); - final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(1); + final TaskSlotTable taskSlotTable = + TaskSlotUtils.createTaskSlotTable(1, EXECUTOR_RESOURCE.getExecutor()); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder().setTaskSlotTable(taskSlotTable).build(); @@ -1837,7 +1858,8 @@ public void testIgnoringSlotRequestsIfNotRegistered() throws Exception { */ @Test public void testReconnectionAttemptIfExplicitlyDisconnected() throws Exception { - final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(1); + final TaskSlotTable taskSlotTable = + TaskSlotUtils.createTaskSlotTable(1, EXECUTOR_RESOURCE.getExecutor()); final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation = new LocalUnresolvedTaskManagerLocation(); final TaskExecutor taskExecutor = @@ -1972,7 +1994,8 @@ public void testRegisterWithDefaultSlotResourceProfile() throws Exception { /** Tests that the {@link TaskExecutor} tries to reconnect if the initial slot report fails. */ @Test public void testInitialSlotReportFailure() throws Exception { - final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(1); + final TaskSlotTable taskSlotTable = + TaskSlotUtils.createTaskSlotTable(1, EXECUTOR_RESOURCE.getExecutor()); final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation = new LocalUnresolvedTaskManagerLocation(); final TaskManagerServices taskManagerServices = @@ -2034,7 +2057,8 @@ public void testInitialSlotReportFailure() throws Exception { /** Tests that offers slots to job master timeout and retry. */ @Test public void testOfferSlotToJobMasterAfterTimeout() throws Exception { - final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(2); + final TaskSlotTable taskSlotTable = + TaskSlotUtils.createTaskSlotTable(2, EXECUTOR_RESOURCE.getExecutor()); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder().setTaskSlotTable(taskSlotTable).build(); @@ -2112,7 +2136,9 @@ public void testOfferSlotToJobMasterAfterTimeout() throws Exception { public void testDisconnectFromJobMasterWhenNewLeader() throws Exception { final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskSlotTable(TaskSlotUtils.createTaskSlotTable(1)) + .setTaskSlotTable( + TaskSlotUtils.createTaskSlotTable( + 1, EXECUTOR_RESOURCE.getExecutor())) .build(); final TaskExecutor taskExecutor = createTaskExecutor(taskManagerServices); @@ -2197,7 +2223,7 @@ public void testLogNotFoundHandling() throws Throwable { new Builder(jobId) .setConfiguration(configuration) .setLocalCommunication(false) - .build()) { + .build(EXECUTOR_RESOURCE.getExecutor())) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); try { CompletableFuture logFuture = @@ -2215,7 +2241,9 @@ public void testLogNotFoundHandling() throws Throwable { @Test(timeout = 10000L) public void testTerminationOnFatalError() throws Throwable { try (TaskSubmissionTestEnvironment env = - new Builder(jobId).setConfiguration(configuration).build()) { + new Builder(jobId) + .setConfiguration(configuration) + .build(EXECUTOR_RESOURCE.getExecutor())) { String testExceptionMsg = "Test exception of fatal error."; env.getTaskExecutor().onFatalError(new Exception(testExceptionMsg)); @@ -2567,7 +2595,9 @@ public void taskExecutorJobServicesCloseClassLoaderLeaseUponClosing() public void testReleaseOfJobResourcesIfJobMasterIsNotCorrect() throws Exception { final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskSlotTable(TaskSlotUtils.createTaskSlotTable(1)) + .setTaskSlotTable( + TaskSlotUtils.createTaskSlotTable( + 1, EXECUTOR_RESOURCE.getExecutor())) .build(); final TestingTaskExecutorPartitionTracker taskExecutorPartitionTracker = @@ -2646,7 +2676,9 @@ public void testReleaseOfJobResourcesIfJobMasterIsNotCorrect() throws Exception public void testReleaseInactiveSlots() throws Exception { final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setTaskSlotTable(TaskSlotUtils.createTaskSlotTable(1)) + .setTaskSlotTable( + TaskSlotUtils.createTaskSlotTable( + 1, EXECUTOR_RESOURCE.getExecutor())) .build(); final TaskExecutor taskExecutor = @@ -2712,7 +2744,8 @@ private TaskExecutorLocalStateStoresManager createTaskExecutorLocalStateStoresMa } private TaskExecutor createTaskExecutor(int numberOFSlots) throws IOException { - final TaskSlotTable taskSlotTable = TaskSlotUtils.createTaskSlotTable(numberOFSlots); + final TaskSlotTable taskSlotTable = + TaskSlotUtils.createTaskSlotTable(numberOFSlots, EXECUTOR_RESOURCE.getExecutor()); final UnresolvedTaskManagerLocation unresolvedTaskManagerLocation = new LocalUnresolvedTaskManagerLocation(); final TaskManagerServices taskManagerServices = @@ -2799,7 +2832,8 @@ private TestingTaskExecutor createTestingTaskExecutor( private TaskExecutorTestingContext createTaskExecutorTestingContext(int numberOfSlots) throws IOException { - return createTaskExecutorTestingContext(TaskSlotUtils.createTaskSlotTable(numberOfSlots)); + return createTaskExecutorTestingContext( + TaskSlotUtils.createTaskSlotTable(numberOfSlots, EXECUTOR_RESOURCE.getExecutor())); } private TaskExecutorTestingContext createTaskExecutorTestingContext( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java index 536b27992d379..a2d299cc84863 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java @@ -36,10 +36,9 @@ import org.apache.flink.runtime.taskmanager.LocalUnresolvedTaskManagerLocation; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.UnresolvedTaskManagerLocation; -import org.apache.flink.testutils.TestingUtils; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import static org.mockito.Mockito.mock; @@ -59,7 +58,6 @@ public class TaskManagerServicesBuilder { private TaskExecutorLocalStateStoresManager taskStateManager; private TaskExecutorStateChangelogStoragesManager taskChangelogStoragesManager; private TaskEventDispatcher taskEventDispatcher; - private ExecutorService ioExecutor; private LibraryCacheManager libraryCacheManager; private long managedMemorySize; private SlotAllocationSnapshotPersistenceService slotAllocationSnapshotPersistenceService; @@ -82,7 +80,6 @@ public TaskManagerServicesBuilder() { RetryingRegistrationConfiguration.defaultConfiguration()); taskStateManager = mock(TaskExecutorLocalStateStoresManager.class); taskChangelogStoragesManager = mock(TaskExecutorStateChangelogStoragesManager.class); - ioExecutor = TestingUtils.defaultExecutor(); libraryCacheManager = TestingLibraryCacheManager.newBuilder().build(); managedMemorySize = MemoryManager.MIN_PAGE_SIZE; this.slotAllocationSnapshotPersistenceService = @@ -144,11 +141,6 @@ public TaskManagerServicesBuilder setTaskChangelogStoragesManager( return this; } - public TaskManagerServicesBuilder setIOExecutorService(ExecutorService ioExecutor) { - this.ioExecutor = ioExecutor; - return this; - } - public TaskManagerServicesBuilder setLibraryCacheManager( LibraryCacheManager libraryCacheManager) { this.libraryCacheManager = libraryCacheManager; @@ -180,7 +172,7 @@ public TaskManagerServices build() { taskStateManager, taskChangelogStoragesManager, taskEventDispatcher, - ioExecutor, + Executors.newSingleThreadScheduledExecutor(), libraryCacheManager, slotAllocationSnapshotPersistenceService); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java index 9913dd9001a94..5ab9a22edda05 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java @@ -60,7 +60,6 @@ import org.apache.flink.runtime.util.ConfigurationParserUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.testutils.TestFileUtils; -import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.Reference; import org.apache.flink.util.concurrent.Executors; @@ -79,6 +78,7 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import static org.mockito.Mockito.mock; @@ -92,8 +92,7 @@ class TaskSubmissionTestEnvironment implements AutoCloseable { private final Time timeout = Time.milliseconds(10000L); private final TestingFatalErrorHandler testingFatalErrorHandler = new TestingFatalErrorHandler(); - private final TimerService timerService = - new DefaultTimerService<>(TestingUtils.defaultExecutor(), timeout.toMilliseconds()); + private final TimerService timerService; private final TestingHighAvailabilityServices haServices; private final TemporaryFolder temporaryFolder; @@ -112,9 +111,15 @@ private TaskSubmissionTestEnvironment( taskManagerActionListeners, @Nullable String metricQueryServiceAddress, TestingRpcService testingRpcService, - ShuffleEnvironment shuffleEnvironment) + ShuffleEnvironment shuffleEnvironment, + ScheduledExecutorService executor) throws Exception { + this.timerService = + new DefaultTimerService<>( + java.util.concurrent.Executors.newSingleThreadScheduledExecutor(), + timeout.toMilliseconds()); + this.haServices = new TestingHighAvailabilityServices(); this.haServices.setResourceManagerLeaderRetriever(new SettableLeaderRetrievalService()); this.haServices.setJobMasterLeaderRetriever(jobId, new SettableLeaderRetrievalService()); @@ -126,7 +131,7 @@ private TaskSubmissionTestEnvironment( final TaskSlotTable taskSlotTable = slotSize > 0 - ? TaskSlotUtils.createTaskSlotTable(slotSize) + ? TaskSlotUtils.createTaskSlotTable(slotSize, executor) : TestingTaskSlotTable.newBuilder() .tryMarkSlotActiveReturns(true) .addTaskReturns(true) @@ -412,7 +417,8 @@ public Builder setResourceID(ResourceID resourceID) { return this; } - public TaskSubmissionTestEnvironment build() throws Exception { + public TaskSubmissionTestEnvironment build(ScheduledExecutorService executorService) + throws Exception { final TestingRpcService testingRpcService = new TestingRpcService(); final ShuffleEnvironment network = optionalShuffleEnvironment.orElseGet( @@ -438,7 +444,8 @@ public TaskSubmissionTestEnvironment build() throws Exception { taskManagerActionListeners, metricQueryServiceAddress, testingRpcService, - network); + network, + executorService); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImplTest.java index 23e692e925a4a..478458fc0919e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotTableImplTest.java @@ -28,11 +28,14 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.taskexecutor.SlotReport; import org.apache.flink.runtime.taskexecutor.SlotStatus; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; import org.apache.flink.util.function.TriFunctionWithException; import org.apache.flink.shaded.guava30.com.google.common.collect.Sets; +import org.junit.ClassRule; import org.junit.Test; import java.util.ArrayList; @@ -41,6 +44,7 @@ import java.util.Iterator; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; @@ -50,6 +54,10 @@ /** Tests for the {@link TaskSlotTable}. */ public class TaskSlotTableImplTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static final Time SLOT_TIMEOUT = Time.seconds(100L); /** Tests that one can can mark allocated slots as active. */ @@ -537,7 +545,7 @@ private static TaskSlotTableImpl createTaskSlotTableAndStart( private static TaskSlotTableImpl createTaskSlotTableAndStart( final int numberOfSlots, final SlotActions slotActions) { final TaskSlotTableImpl taskSlotTable = - TaskSlotUtils.createTaskSlotTable(numberOfSlots); + TaskSlotUtils.createTaskSlotTable(numberOfSlots, EXECUTOR_RESOURCE.getExecutor()); taskSlotTable.start(slotActions, ComponentMainThreadExecutorServiceAdapter.forMainThread()); return taskSlotTable; } @@ -545,7 +553,8 @@ private static TaskSlotTableImpl createTaskSlotTableAndStart( private static TaskSlotTableImpl createTaskSlotTableAndStart( final int numberOfSlots, TimerService timerService) { final TaskSlotTableImpl taskSlotTable = - TaskSlotUtils.createTaskSlotTable(numberOfSlots, timerService); + TaskSlotUtils.createTaskSlotTable( + numberOfSlots, timerService, EXECUTOR_RESOURCE.getExecutor()); taskSlotTable.start( new TestingSlotActionsBuilder().build(), ComponentMainThreadExecutorServiceAdapter.forMainThread()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotUtils.java index 6255316bac8ae..9316cc85afab5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/slot/TaskSlotUtils.java @@ -23,8 +23,9 @@ import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.memory.MemoryManager; -import org.apache.flink.testutils.TestingUtils; -import org.apache.flink.util.concurrent.Executors; + +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; /** Testing utility and factory methods for {@link TaskSlotTable} and {@link TaskSlot}s. */ public enum TaskSlotUtils { @@ -42,25 +43,30 @@ public enum TaskSlotUtils { .build(); public static TaskSlotTableImpl createTaskSlotTable( - int numberOfSlots) { - return createTaskSlotTable(numberOfSlots, createDefaultTimerService()); + int numberOfSlots, ScheduledExecutorService executorService) { + return createTaskSlotTable( + numberOfSlots, createDefaultTimerService(DEFAULT_SLOT_TIMEOUT), executorService); } public static TaskSlotTable createTaskSlotTable( - int numberOfSlots, Time timeout) { + int numberOfSlots, Time timeout, ScheduledExecutorService executorService) { return createTaskSlotTable( - numberOfSlots, createDefaultTimerService(timeout.toMilliseconds())); + numberOfSlots, + createDefaultTimerService(timeout.toMilliseconds()), + executorService); } public static TaskSlotTableImpl createTaskSlotTable( - int numberOfSlots, TimerService timerService) { + int numberOfSlots, + TimerService timerService, + ScheduledExecutorService executorService) { return new TaskSlotTableImpl<>( numberOfSlots, createTotalResourceProfile(numberOfSlots), DEFAULT_RESOURCE_PROFILE, MemoryManager.MIN_PAGE_SIZE, timerService, - Executors.newDirectExecutorService()); + executorService); } public static ResourceProfile createTotalResourceProfile(int numberOfSlots) { @@ -71,11 +77,8 @@ public static ResourceProfile createTotalResourceProfile(int numberOfSlots) { return result; } - public static TimerService createDefaultTimerService() { - return createDefaultTimerService(DEFAULT_SLOT_TIMEOUT); - } - public static TimerService createDefaultTimerService(long shutdownTimeout) { - return new DefaultTimerService<>(TestingUtils.defaultExecutor(), shutdownTimeout); + return new DefaultTimerService<>( + Executors.newSingleThreadScheduledExecutor(), shutdownTimeout); } } 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 bb5c6bf006f66..aba82596619b4 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 @@ -43,9 +43,11 @@ import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.junit5.InjectMiniCluster; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorExtension; import org.apache.flink.types.LongValue; import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -53,6 +55,7 @@ import java.time.Duration; import java.util.Arrays; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import static org.apache.flink.runtime.io.network.buffer.LocalBufferPoolDestroyTest.isInBlockingBufferRequest; @@ -63,6 +66,10 @@ @ExtendWith({TestLoggerExtension.class}) public class TaskCancelAsyncProducerConsumerITCase { + @RegisterExtension + static final TestExecutorExtension EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorExtension(); + // The Exceptions thrown by the producer/consumer Threads private static volatile Exception ASYNC_PRODUCER_EXCEPTION; private static volatile Exception ASYNC_CONSUMER_EXCEPTION; @@ -123,7 +130,7 @@ public void testCancelAsyncProducerAndConsumer(@InjectMiniCluster MiniCluster fl Time.milliseconds(10), deadline, status -> status == JobStatus.RUNNING, - TestingUtils.defaultScheduledExecutor()) + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor())) .get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); boolean producerBlocked = false; @@ -177,7 +184,7 @@ public void testCancelAsyncProducerAndConsumer(@InjectMiniCluster MiniCluster fl Time.milliseconds(10), deadline, status -> status == JobStatus.CANCELED, - TestingUtils.defaultScheduledExecutor()) + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor())) .get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); // Verify the expected Exceptions diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index 6cf0dd7ff1839..543c875d787b1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -49,6 +49,8 @@ import org.apache.flink.runtime.state.CheckpointStorageLocationReference; import org.apache.flink.runtime.taskexecutor.PartitionProducerStateChecker; import org.apache.flink.runtime.util.NettyShuffleDescriptorBuilder; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.apache.flink.util.WrappingRuntimeException; @@ -73,6 +75,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; @@ -105,6 +108,10 @@ public class TaskTest extends TestLogger { private ShuffleEnvironment shuffleEnvironment; + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); private static boolean wasCleanedUp = false; @@ -132,19 +139,28 @@ public void teardown() throws Exception { @Test public void testCleanupWhenRestoreFails() throws Exception { - createTaskBuilder().setInvokable(InvokableWithExceptionInRestore.class).build().run(); + createTaskBuilder() + .setInvokable(InvokableWithExceptionInRestore.class) + .build(Executors.directExecutor()) + .run(); assertTrue(wasCleanedUp); } @Test public void testCleanupWhenInvokeFails() throws Exception { - createTaskBuilder().setInvokable(InvokableWithExceptionInInvoke.class).build().run(); + createTaskBuilder() + .setInvokable(InvokableWithExceptionInInvoke.class) + .build(Executors.directExecutor()) + .run(); assertTrue(wasCleanedUp); } @Test public void testCleanupWhenCancelledAfterRestore() throws Exception { - Task task = createTaskBuilder().setInvokable(InvokableBlockingInRestore.class).build(); + Task task = + createTaskBuilder() + .setInvokable(InvokableBlockingInRestore.class) + .build(Executors.directExecutor()); task.startTaskThread(); awaitLatch.await(); task.cancelExecution(); @@ -154,7 +170,10 @@ public void testCleanupWhenCancelledAfterRestore() throws Exception { @Test public void testCleanupWhenAfterInvokeSucceeded() throws Exception { - createTaskBuilder().setInvokable(TestInvokableCorrect.class).build().run(); + createTaskBuilder() + .setInvokable(TestInvokableCorrect.class) + .build(Executors.directExecutor()) + .run(); assertTrue(wasCleanedUp); } @@ -173,7 +192,7 @@ public void updateTaskExecutionState( } } }) - .build() + .build(Executors.directExecutor()) .run(); assertTrue(wasCleanedUp); } @@ -185,7 +204,7 @@ public void testRegularExecution() throws Exception { createTaskBuilder() .setInvokable(TestInvokableCorrect.class) .setTaskManagerActions(taskManagerActions) - .build(); + .build(Executors.directExecutor()); // task should be new and perfect assertEquals(ExecutionState.CREATED, task.getExecutionState()); @@ -209,7 +228,7 @@ public void testRegularExecution() throws Exception { @Test public void testCancelRightAway() throws Exception { - final Task task = createTaskBuilder().build(); + final Task task = createTaskBuilder().build(Executors.directExecutor()); task.cancelExecution(); assertEquals(ExecutionState.CANCELING, task.getExecutionState()); @@ -224,7 +243,7 @@ public void testCancelRightAway() throws Exception { @Test public void testFailExternallyRightAway() throws Exception { - final Task task = createTaskBuilder().build(); + final Task task = createTaskBuilder().build(Executors.directExecutor()); task.failExternally(new Exception("fail externally")); assertEquals(ExecutionState.FAILED, task.getExecutionState()); @@ -249,7 +268,7 @@ public void testLibraryCacheRegistrationFailed() throws Exception { throw testException; }) .build()) - .build(); + .build(Executors.directExecutor()); // task should be new and perfect assertEquals(ExecutionState.CREATED, task.getExecutionState()); @@ -315,7 +334,7 @@ private void testExecutionFailsInNetworkRegistration( .setPartitionProducerStateChecker(partitionProducerStateChecker) .setResultPartitions(resultPartitions) .setInputGates(inputGates) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); // shut down the network to make the following task registration failure shuffleEnvironment.close(); @@ -339,7 +358,7 @@ public void testInvokableInstantiationFailed() throws Exception { createTaskBuilder() .setTaskManagerActions(taskManagerActions) .setInvokable(InvokableNonInstantiable.class) - .build(); + .build(Executors.directExecutor()); // should fail task.run(); @@ -362,7 +381,7 @@ public void testExecutionFailsInRestore() throws Exception { createTaskBuilder() .setInvokable(InvokableWithExceptionInRestore.class) .setTaskManagerActions(taskManagerActions) - .build(); + .build(Executors.directExecutor()); task.run(); @@ -384,7 +403,7 @@ public void testExecutionFailsInInvoke() throws Exception { createTaskBuilder() .setInvokable(InvokableWithExceptionInInvoke.class) .setTaskManagerActions(taskManagerActions) - .build(); + .build(Executors.directExecutor()); task.run(); @@ -407,7 +426,7 @@ public void testFailWithWrappedException() throws Exception { createTaskBuilder() .setInvokable(FailingInvokableWithChainedException.class) .setTaskManagerActions(taskManagerActions) - .build(); + .build(Executors.directExecutor()); task.run(); @@ -430,7 +449,7 @@ public void testCancelDuringRestore() throws Exception { createTaskBuilder() .setInvokable(InvokableBlockingInRestore.class) .setTaskManagerActions(taskManagerActions) - .build(); + .build(Executors.directExecutor()); // run the task asynchronous task.startTaskThread(); @@ -460,7 +479,7 @@ public void testCancelDuringInvoke() throws Exception { createTaskBuilder() .setInvokable(InvokableBlockingInInvoke.class) .setTaskManagerActions(taskManagerActions) - .build(); + .build(Executors.directExecutor()); // run the task asynchronous task.startTaskThread(); @@ -491,7 +510,7 @@ public void testFailExternallyDuringRestore() throws Exception { createTaskBuilder() .setInvokable(InvokableBlockingInRestore.class) .setTaskManagerActions(taskManagerActions) - .build(); + .build(Executors.directExecutor()); // run the task asynchronous task.startTaskThread(); @@ -519,7 +538,7 @@ public void testFailExternallyDuringInvoke() throws Exception { createTaskBuilder() .setInvokable(InvokableBlockingInInvoke.class) .setTaskManagerActions(taskManagerActions) - .build(); + .build(Executors.directExecutor()); // run the task asynchronous task.startTaskThread(); @@ -548,7 +567,7 @@ public void testCanceledAfterExecutionFailedInInvoke() throws Exception { createTaskBuilder() .setInvokable(InvokableWithExceptionInInvoke.class) .setTaskManagerActions(taskManagerActions) - .build(); + .build(Executors.directExecutor()); task.run(); @@ -572,7 +591,7 @@ public void testExecutionFailsAfterCanceling() throws Exception { createTaskBuilder() .setInvokable(InvokableWithExceptionOnTrigger.class) .setTaskManagerActions(taskManagerActions) - .build(); + .build(Executors.directExecutor()); // run the task asynchronous task.startTaskThread(); @@ -605,7 +624,7 @@ public void testExecutionFailsAfterTaskMarkedFailed() throws Exception { createTaskBuilder() .setInvokable(InvokableWithExceptionOnTrigger.class) .setTaskManagerActions(taskManagerActions) - .build(); + .build(Executors.directExecutor()); // run the task asynchronous task.startTaskThread(); @@ -636,7 +655,7 @@ public void testCancelTaskException() throws Exception { final Task task = createTaskBuilder() .setInvokable(InvokableWithCancelTaskExceptionInInvoke.class) - .build(); + .build(Executors.directExecutor()); // Cause CancelTaskException. triggerLatch.trigger(); @@ -651,7 +670,7 @@ public void testCancelTaskExceptionAfterTaskMarkedFailed() throws Exception { final Task task = createTaskBuilder() .setInvokable(InvokableWithCancelTaskExceptionInInvoke.class) - .build(); + .build(Executors.directExecutor()); task.startTaskThread(); @@ -691,7 +710,10 @@ public void testOnPartitionStateUpdateWhileDeploying() throws Exception { public void testOnPartitionStateUpdate(ExecutionState initialTaskState) throws Exception { final ResultPartitionID partitionId = new ResultPartitionID(); - final Task task = createTaskBuilder().setInvokable(InvokableBlockingInInvoke.class).build(); + final Task task = + createTaskBuilder() + .setInvokable(InvokableBlockingInInvoke.class) + .build(Executors.directExecutor()); RemoteChannelStateChecker checker = new RemoteChannelStateChecker(partitionId, "test task"); @@ -759,8 +781,7 @@ public void testTriggerPartitionStateUpdate() throws Exception { .setInvokable(InvokableBlockingInInvoke.class) .setConsumableNotifier(consumableNotifier) .setPartitionProducerStateChecker(partitionChecker) - .setExecutor(Executors.directExecutor()) - .build(); + .build(Executors.directExecutor()); TestTaskBuilder.setTaskState(task, ExecutionState.RUNNING); final CompletableFuture promise = new CompletableFuture<>(); @@ -791,8 +812,7 @@ public void testTriggerPartitionStateUpdate() throws Exception { .setInvokable(InvokableBlockingInInvoke.class) .setConsumableNotifier(consumableNotifier) .setPartitionProducerStateChecker(partitionChecker) - .setExecutor(Executors.directExecutor()) - .build(); + .build(Executors.directExecutor()); TestTaskBuilder.setTaskState(task, ExecutionState.RUNNING); final CompletableFuture promise = new CompletableFuture<>(); @@ -827,8 +847,7 @@ public void testTriggerPartitionStateUpdate() throws Exception { .setInvokable(InvokableBlockingInInvoke.class) .setConsumableNotifier(consumableNotifier) .setPartitionProducerStateChecker(partitionChecker) - .setExecutor(Executors.directExecutor()) - .build(); + .build(Executors.directExecutor()); try { task.startTaskThread(); @@ -872,8 +891,7 @@ public void testTriggerPartitionStateUpdate() throws Exception { .setInvokable(InvokableBlockingInInvoke.class) .setConsumableNotifier(consumableNotifier) .setPartitionProducerStateChecker(partitionChecker) - .setExecutor(Executors.directExecutor()) - .build(); + .build(Executors.directExecutor()); try { task.startTaskThread(); @@ -923,7 +941,7 @@ public void testWatchDogInterruptsTask() throws Exception { .setInvokable(InvokableBlockingInCancel.class) .setTaskManagerConfig(config) .setTaskManagerActions(taskManagerActions) - .build(); + .build(Executors.directExecutor()); task.startTaskThread(); @@ -951,7 +969,7 @@ public void testInterruptibleSharedLockInInvokeAndCancel() throws Exception { .setInvokable(InvokableInterruptibleSharedLockInInvokeAndCancel.class) .setTaskManagerConfig(config) .setTaskManagerActions(taskManagerActions) - .build(); + .build(Executors.directExecutor()); task.startTaskThread(); @@ -982,7 +1000,7 @@ public void testFatalErrorAfterUnInterruptibleInvoke() throws Exception { .setInvokable(InvokableUnInterruptibleBlockingInvoke.class) .setTaskManagerConfig(config) .setTaskManagerActions(taskManagerActions) - .build(); + .build(Executors.directExecutor()); try { task.startTaskThread(); @@ -1022,7 +1040,7 @@ public void testFatalErrorOnCanceling() throws Exception { .setInvokable(InvokableBlockingWithTrigger.class) .setTaskManagerConfig(config) .setTaskManagerActions(taskManagerActions) - .build()); + .build(Executors.directExecutor())); final Class fatalErrorType = OutOfMemoryError.class; doThrow(fatalErrorType) @@ -1063,7 +1081,7 @@ public void testTaskConfig() throws Exception { .setInvokable(InvokableBlockingInInvoke.class) .setTaskManagerConfig(config) .setExecutionConfig(executionConfig) - .build(); + .build(Executors.directExecutor()); assertEquals(interval, task.getTaskCancellationInterval()); assertEquals(timeout, task.getTaskCancellationTimeout()); @@ -1087,7 +1105,7 @@ public void testTerminationFutureCompletesOnNormalExecution() throws Exception { createTaskBuilder() .setInvokable(InvokableBlockingWithTrigger.class) .setTaskManagerActions(new NoOpTaskManagerActions()) - .build(); + .build(Executors.directExecutor()); // run the task asynchronous task.startTaskThread(); @@ -1110,7 +1128,7 @@ public void testTerminationFutureCompletesOnImmediateCancellation() throws Excep createTaskBuilder() .setInvokable(InvokableBlockingInInvoke.class) .setTaskManagerActions(new NoOpTaskManagerActions()) - .build(); + .build(Executors.directExecutor()); task.cancelExecution(); @@ -1130,7 +1148,7 @@ public void testTerminationFutureCompletesOnErrorInInvoke() throws Exception { createTaskBuilder() .setInvokable(InvokableWithExceptionInInvoke.class) .setTaskManagerActions(new NoOpTaskManagerActions()) - .build(); + .build(Executors.directExecutor()); // run the task asynchronous task.startTaskThread(); @@ -1142,7 +1160,7 @@ public void testTerminationFutureCompletesOnErrorInInvoke() throws Exception { @Test public void testNoBackPressureIfTaskNotStarted() throws Exception { - final Task task = createTaskBuilder().build(); + final Task task = createTaskBuilder().build(Executors.directExecutor()); assertFalse(task.isBackPressured()); } @@ -1153,7 +1171,7 @@ public void testDeclineCheckpoint() throws Exception { createTaskBuilder() .setInvokable(InvokableDeclingingCheckpoints.class) .setCheckpointResponder(testCheckpointResponder) - .build(); + .build(Executors.directExecutor()); assertCheckpointDeclined( task, testCheckpointResponder, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java index 527e16b8430f0..698c4159b53ec 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TestTaskBuilder.java @@ -53,7 +53,6 @@ import org.apache.flink.runtime.taskexecutor.PartitionProducerStateChecker; import org.apache.flink.runtime.taskexecutor.TestGlobalAggregateManager; import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; -import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; @@ -78,7 +77,6 @@ public final class TestTaskBuilder { new NoOpPartitionProducerStateChecker(); private final ShuffleEnvironment shuffleEnvironment; private KvStateService kvStateService = new KvStateService(new KvStateRegistry(), null, null); - private Executor executor = TestingUtils.defaultExecutor(); private Configuration taskManagerConfig = new Configuration(); private Configuration taskConfig = new Configuration(); private ExecutionConfig executionConfig = new ExecutionConfig(); @@ -129,11 +127,6 @@ public TestTaskBuilder setKvStateService(KvStateService kvStateService) { return this; } - public TestTaskBuilder setExecutor(Executor executor) { - this.executor = executor; - return this; - } - public TestTaskBuilder setTaskManagerConfig(Configuration taskManagerConfig) { this.taskManagerConfig = taskManagerConfig; return this; @@ -192,7 +185,7 @@ public TestTaskBuilder setCheckpointResponder(TestCheckpointResponder testCheckp return this; } - public Task build() throws Exception { + public Task build(Executor executor) throws Exception { final JobVertexID jobVertexId = new JobVertexID(); final SerializedValue serializedExecutionConfig = diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java index 6daaee8b3a08a..0b2cdd9a8bd50 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java @@ -40,8 +40,11 @@ import org.apache.flink.streaming.runtime.tasks.SourceStreamTask; import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.tasks.StreamTaskTest; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import java.io.Serializable; @@ -50,11 +53,15 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.ScheduledExecutorService; import static org.junit.Assert.assertEquals; /** This test secures the lifecycle of AbstractUdfStreamOperator, including it's UDF handling. */ public class AbstractUdfStreamOperatorLifecycleTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); private static final List EXPECTED_CALL_ORDER_FULL = Arrays.asList( @@ -156,7 +163,11 @@ public void testLifeCycleFull() throws Exception { try (ShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build()) { Task task = StreamTaskTest.createTask( - SourceStreamTask.class, shuffleEnvironment, cfg, taskManagerConfig); + SourceStreamTask.class, + shuffleEnvironment, + cfg, + taskManagerConfig, + EXECUTOR_RESOURCE.getExecutor()); task.startTaskThread(); @@ -183,7 +194,11 @@ public void testLifeCycleCancel() throws Exception { try (ShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build()) { Task task = StreamTaskTest.createTask( - SourceStreamTask.class, shuffleEnvironment, cfg, taskManagerConfig); + SourceStreamTask.class, + shuffleEnvironment, + cfg, + taskManagerConfig, + EXECUTOR_RESOURCE.getExecutor()); task.startTaskThread(); LifecycleTrackingStreamSource.runStarted.await(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java index 1484d67459013..c2c72a820c9c5 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/partitioner/RescalePartitionerTest.java @@ -34,8 +34,11 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.Collector; +import org.junit.ClassRule; import org.junit.Test; import java.util.HashMap; @@ -43,6 +46,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ScheduledExecutorService; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -51,6 +55,9 @@ /** Tests for {@link RescalePartitioner}. */ @SuppressWarnings("serial") public class RescalePartitionerTest extends StreamPartitionerTest { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); @Override public StreamPartitioner createPartitioner() { @@ -121,7 +128,7 @@ public void flatMap( TestingDefaultExecutionGraphBuilder.newBuilder() .setVertexParallelismStore( SchedulerBase.computeVertexParallelismStore(jobGraph)) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); try { eg.attachJobGraph(jobVertices); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationTest.java index d14680f49c668..7589224b69a90 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskCancellationTest.java @@ -31,11 +31,15 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; import org.junit.Test; import java.io.Closeable; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.STRING_TYPE_INFO; @@ -45,6 +49,9 @@ /** Tests for the StreamTask cancellation. */ public class StreamTaskCancellationTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); @Test public void testDoNotInterruptWhileClosing() throws Exception { @@ -91,7 +98,11 @@ public void testCanceleablesCanceledOnCancelTaskError() throws Exception { Task task = createTask( - CancelFailingTask.class, shuffleEnvironment, cfg, new Configuration()); + CancelFailingTask.class, + shuffleEnvironment, + cfg, + new Configuration(), + EXECUTOR_RESOURCE.getExecutor()); // start the task and wait until it runs // execution state RUNNING is not enough, we need to wait until the stream task's run() @@ -217,7 +228,11 @@ public void testCancelTaskExceptionHandling() throws Exception { new NettyShuffleEnvironmentBuilder().build()) { Task task = createTask( - CancelThrowingTask.class, shuffleEnvironment, cfg, new Configuration()); + CancelThrowingTask.class, + shuffleEnvironment, + cfg, + new Configuration(), + EXECUTOR_RESOURCE.getExecutor()); task.startTaskThread(); task.getExecutingThread().join(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java index 9a87d040a1b34..60f4bfbf704fe 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java @@ -78,12 +78,14 @@ import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.FlinkException; import org.apache.flink.util.SerializedValue; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.Executors; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Test; import javax.annotation.Nonnull; @@ -94,6 +96,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.FutureTask; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -106,6 +109,10 @@ /** Tests for the StreamTask termination. */ public class StreamTaskTerminationTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static final OneShotLatch RUN_LATCH = new OneShotLatch(); private static final AtomicBoolean SNAPSHOT_HAS_STARTED = new AtomicBoolean(); private static final OneShotLatch CLEANUP_LATCH = new OneShotLatch(); @@ -186,7 +193,7 @@ public void testConcurrentAsyncCheckpointCannotFailFinishedStreamTask() throws E Executors.directExecutor()); CompletableFuture taskRun = - CompletableFuture.runAsync(() -> task.run(), TestingUtils.defaultExecutor()); + CompletableFuture.runAsync(() -> task.run(), EXECUTOR_RESOURCE.getExecutor()); // wait until the stream task started running RUN_LATCH.await(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index f2bac03e2e33d..6e51e7a61d711 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -128,6 +128,8 @@ import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; import org.apache.flink.streaming.util.MockStreamConfig; import org.apache.flink.streaming.util.MockStreamTaskBuilder; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.CloseableIterable; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FatalExitExceptionHandler; @@ -145,6 +147,7 @@ import org.hamcrest.CoreMatchers; import org.hamcrest.Matchers; import org.junit.Assert; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -168,9 +171,11 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.RunnableFuture; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; @@ -224,6 +229,9 @@ /** Tests for {@link StreamTask}. */ public class StreamTaskTest extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); @Rule public ExpectedException thrown = ExpectedException.none(); @@ -450,7 +458,7 @@ public void testEarlyCanceling() throws Exception { .setInvokable(SourceStreamTask.class) .setTaskConfig(cfg.getConfiguration()) .setTaskManagerActions(taskManagerActions) - .build(); + .build(EXECUTOR_RESOURCE.getExecutor()); final TaskExecutionState state = new TaskExecutionState(task.getExecutionId(), ExecutionState.RUNNING); @@ -489,7 +497,8 @@ public void testStateBackendLoadingAndClosing() throws Exception { StateBackendTestSource.class, shuffleEnvironment, cfg, - taskManagerConfig); + taskManagerConfig, + EXECUTOR_RESOURCE.getExecutor()); StateBackendTestSource.fail = false; task.startTaskThread(); @@ -530,7 +539,8 @@ public void testStateBackendClosingOnFailure() throws Exception { StateBackendTestSource.class, shuffleEnvironment, cfg, - taskManagerConfig); + taskManagerConfig, + EXECUTOR_RESOURCE.getExecutor()); StateBackendTestSource.fail = true; task.startTaskThread(); @@ -1205,7 +1215,7 @@ public void updateTaskExecutionState(TaskExecutionState taskExecutionState) { env -> taskBuilderWithConfiguredRecordWriter(env) .setTaskManagerActions(taskManagerActions) - .build()); + .build(EXECUTOR_RESOURCE.getExecutor())); } @Test @@ -1217,7 +1227,7 @@ public void testFailInEndOfConstructor() throws Exception { env -> taskBuilderWithConfiguredRecordWriter(env) .setTaskManagerConfig(conf) - .build()); + .build(EXECUTOR_RESOURCE.getExecutor())); } private void testRecordWriterClosedOnError( @@ -1583,7 +1593,12 @@ public void testTaskAvoidHangingAfterSnapshotStateThrownException() throws Excep try (NettyShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build()) { Task task = - createTask(SourceStreamTask.class, shuffleEnvironment, cfg, taskManagerConfig); + createTask( + SourceStreamTask.class, + shuffleEnvironment, + cfg, + taskManagerConfig, + EXECUTOR_RESOURCE.getExecutor()); // when: Task starts task.startTaskThread(); @@ -2097,14 +2112,15 @@ public static Task createTask( Class invokable, ShuffleEnvironment shuffleEnvironment, StreamConfig taskConfig, - Configuration taskManagerConfig) + Configuration taskManagerConfig, + Executor executor) throws Exception { return new TestTaskBuilder(shuffleEnvironment) .setTaskManagerConfig(taskManagerConfig) .setInvokable(invokable) .setTaskConfig(taskConfig.getConfiguration()) - .build(); + .build(executor); } // ------------------------------------------------------------------------ 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 aa7f3334f8b4a..097ea98fd6da3 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 @@ -43,9 +43,11 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; import org.junit.Before; import org.junit.ClassRule; @@ -58,6 +60,7 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import static org.apache.flink.test.util.TestUtils.submitJobAndWaitForResult; @@ -67,6 +70,10 @@ public class AccumulatorLiveITCase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(AccumulatorLiveITCase.class); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + // name of user accumulator private static final String ACCUMULATOR_NAME = "test"; @@ -199,7 +206,7 @@ private static void verifyResults(JobGraph jobGraph, Deadline deadline, ClusterC && accumulators.containsKey(ACCUMULATOR_NAME) && (int) accumulators.get(ACCUMULATOR_NAME) == NUM_ITERATIONS, - TestingUtils.defaultScheduledExecutor()) + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor())) .get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java index bf75557cf7568..5962cc3d47175 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java @@ -54,10 +54,12 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FixedRetryStrategy; import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; import org.junit.AfterClass; import org.junit.Before; @@ -79,6 +81,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -91,6 +94,10 @@ @RunWith(Parameterized.class) public class RescalingITCase extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static final int numTaskManagers = 2; private static final int slotsPerTaskManager = 2; private static final int numSlots = numTaskManagers * slotsPerTaskManager; @@ -559,7 +566,7 @@ public void testSavepointRescalingPartitionedOperatorState( (int) deadline.timeLeft().getSeconds() / 10, Duration.ofSeconds(10)), (throwable) -> true, - TestingUtils.defaultScheduledExecutor()); + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor())); final String savepointPath = savepointPathFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index 29b9c1c218810..5ebd7ec334871 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -91,6 +91,7 @@ import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.testutils.EntropyInjectingTestFileSystem; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.testutils.junit.SharedObjects; import org.apache.flink.testutils.junit.SharedReference; import org.apache.flink.util.Collector; @@ -98,12 +99,14 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.TypeSafeDiagnosingMatcher; import org.junit.Assert; import org.junit.Before; +import org.junit.ClassRule; import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -133,6 +136,7 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.function.BiFunction; import java.util.stream.Collectors; @@ -161,6 +165,10 @@ public class SavepointITCase extends TestLogger { private static final Logger LOG = LoggerFactory.getLogger(SavepointITCase.class); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Rule public final TemporaryFolder folder = new TemporaryFolder(); private File checkpointDir; @@ -706,7 +714,7 @@ private void restoreJobAndVerifyState( Time.milliseconds(50), Deadline.now().plus(Duration.ofSeconds(30)), status -> status == JobStatus.CANCELED, - TestingUtils.defaultScheduledExecutor()); + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor())); postCancelChecks.check(cluster); } finally { diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java index c41001f674220..0de6d84bc4ec6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java @@ -51,6 +51,7 @@ import org.apache.flink.runtime.testutils.ZooKeeperTestUtils; import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.Collector; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; @@ -60,6 +61,7 @@ import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -74,6 +76,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -101,6 +104,10 @@ public class JobManagerHAProcessFailureRecoveryITCase extends TestLogger { private static final Duration TEST_TIMEOUT = Duration.ofMinutes(5); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); @BeforeClass @@ -289,7 +296,9 @@ public void testDispatcherProcessFailure() throws Exception { highAvailabilityServices = HighAvailabilityServicesUtils.createAvailableOrEmbeddedServices( - config, TestingUtils.defaultExecutor(), NoOpFatalErrorHandler.INSTANCE); + config, + EXECUTOR_RESOURCE.getExecutor(), + NoOpFatalErrorHandler.INSTANCE); final PluginManager pluginManager = PluginUtils.createPluginManagerFromRootFolder(config); 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 fd6bc9c9fdebd..02dd190f4fe77 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 @@ -57,10 +57,12 @@ import org.apache.flink.test.util.TestProcessBuilder; import org.apache.flink.test.util.TestProcessBuilder.TestProcess; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; import org.junit.Assume; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -85,6 +87,10 @@ public class ProcessFailureCancelingITCase extends TestLogger { private static final String TASK_DEPLOYED_MARKER = "deployed"; private static final Duration TIMEOUT = Duration.ofMinutes(2); + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + @Rule public final BlobServerResource blobServerResource = new BlobServerResource(); @Rule public final ZooKeeperResource zooKeeperResource = new ZooKeeperResource(); @@ -127,7 +133,7 @@ public void testCancelingOnProcessFailure() throws Throwable { StandaloneResourceManagerFactory.getInstance()); DispatcherResourceManagerComponent dispatcherResourceManagerComponent = null; - final ScheduledExecutorService ioExecutor = TestingUtils.defaultExecutor(); + final ScheduledExecutorService ioExecutor = EXECUTOR_RESOURCE.getExecutor(); final HighAvailabilityServices haServices = HighAvailabilityServicesUtils.createHighAvailabilityServices( config, 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 0ddf11ea33b69..bc3a227952e6d 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 @@ -35,10 +35,14 @@ import org.apache.flink.streaming.api.graph.StreamingJobGraphGenerator; import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.concurrent.ScheduledExecutor; +import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -47,6 +51,7 @@ import java.net.URL; import java.time.Duration; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -69,6 +74,10 @@ */ public abstract class AbstractOperatorRestoreTestBase extends TestLogger { + @ClassRule + public static final TestExecutorResource EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorResource(); + private static final int NUM_TMS = 1; private static final int NUM_SLOTS_PER_TM = 4; private static final Duration TEST_TIMEOUT = Duration.ofSeconds(10000L); @@ -97,6 +106,8 @@ public abstract class AbstractOperatorRestoreTestBase extends TestLogger { .build()); private final boolean allowNonRestoredState; + private final ScheduledExecutor scheduledExecutor = + new ScheduledExecutorServiceAdapter(EXECUTOR_RESOURCE.getExecutor()); protected AbstractOperatorRestoreTestBase() { this(true); @@ -140,7 +151,7 @@ private String migrateJob(ClusterClient clusterClient, Deadline deadline) thr Time.milliseconds(50), deadline, (jobStatus) -> jobStatus == JobStatus.RUNNING, - TestingUtils.defaultScheduledExecutor()); + scheduledExecutor); assertEquals( JobStatus.RUNNING, jobRunningFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); @@ -179,7 +190,7 @@ private String migrateJob(ClusterClient clusterClient, Deadline deadline) thr Time.milliseconds(50), deadline, (jobStatus) -> jobStatus == JobStatus.CANCELED, - TestingUtils.defaultScheduledExecutor()); + scheduledExecutor); assertEquals( JobStatus.CANCELED, jobCanceledFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); @@ -203,7 +214,7 @@ private void restoreJob(ClusterClient clusterClient, Deadline deadline, Strin Time.milliseconds(50), deadline, (jobStatus) -> jobStatus == JobStatus.FINISHED, - TestingUtils.defaultScheduledExecutor()); + scheduledExecutor); assertEquals( JobStatus.FINISHED, jobStatusFuture.get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); 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 2f35b98e208ef..af2dc2e66468a 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 @@ -37,7 +37,7 @@ import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerInfo; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersHeaders; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersInfo; -import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.util.concurrent.Executors; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; @@ -125,7 +125,7 @@ public void testFlinkContainerMemory() throws Exception { final ApplicationId clusterId = clusterClient.getClusterId(); final RestClient restClient = - new RestClient(configuration, TestingUtils.defaultExecutor()); + new RestClient(configuration, Executors.directExecutor()); try { final ApplicationReport applicationReport = diff --git a/tools/maven/suppressions-runtime.xml b/tools/maven/suppressions-runtime.xml index 404507c95d4f0..74240717738f5 100644 --- a/tools/maven/suppressions-runtime.xml +++ b/tools/maven/suppressions-runtime.xml @@ -138,6 +138,6 @@ under the License. files="(.*)runtime[/\\]zookeeper[/\\](.*)" checks="RedundantModifier|JavadocParagraph|JavadocType|JavadocStyle|MemberNameCheck|LocalFinalVariableName|LocalVariableName|UpperEll|reliefPattern|EmptyStatement|EmptyLineSeparator"/>