diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java index f1d08e184ac6e..6df7118aaa1dd 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherGatewayServiceFactory.java @@ -26,11 +26,13 @@ import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobPersistenceComponents; import org.apache.flink.runtime.dispatcher.runner.AbstractDispatcherLeaderProcess; import org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherGatewayService; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphWriter; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.FlinkRuntimeException; @@ -83,7 +85,9 @@ public ApplicationDispatcherGatewayServiceFactory( public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( DispatcherId fencingToken, Collection recoveredJobs, - JobGraphWriter jobGraphWriter) { + Collection recoveredDirtyJobResults, + JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore) { final List recoveredJobIds = getRecoveredJobIds(recoveredJobs); @@ -94,6 +98,7 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( rpcService, fencingToken, recoveredJobs, + recoveredDirtyJobResults, (dispatcherGateway, scheduledExecutor, errorHandler) -> new ApplicationDispatcherBootstrap( application, @@ -102,8 +107,8 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( dispatcherGateway, scheduledExecutor, errorHandler), - PartialDispatcherServicesWithJobGraphStore.from( - partialDispatcherServices, jobGraphWriter)); + PartialDispatcherServicesWithJobPersistenceComponents.from( + partialDispatcherServices, jobGraphWriter, jobResultStore)); } catch (Exception e) { throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e); } diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherLeaderProcessFactoryFactory.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherLeaderProcessFactoryFactory.java index 2c1ac37187f23..03b94a88ef9e2 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherLeaderProcessFactoryFactory.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherLeaderProcessFactoryFactory.java @@ -26,7 +26,7 @@ import org.apache.flink.runtime.dispatcher.runner.DispatcherLeaderProcessFactory; import org.apache.flink.runtime.dispatcher.runner.DispatcherLeaderProcessFactoryFactory; import org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcessFactory; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -59,7 +59,7 @@ private ApplicationDispatcherLeaderProcessFactoryFactory( @Override public DispatcherLeaderProcessFactory createFactory( - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices, @@ -74,7 +74,10 @@ public DispatcherLeaderProcessFactory createFactory( partialDispatcherServices); return new SessionDispatcherLeaderProcessFactory( - dispatcherServiceFactory, jobGraphStoreFactory, ioExecutor, fatalErrorHandler); + dispatcherServiceFactory, + jobPersistenceComponentFactory, + ioExecutor, + fatalErrorHandler); } public static ApplicationDispatcherLeaderProcessFactoryFactory create( 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 8c0067b279e49..c20a959db4894 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 @@ -26,11 +26,14 @@ import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.ProgramInvocationException; import org.apache.flink.client.testjar.BlockingJob; +import org.apache.flink.client.testjar.ErrorHandlingSubmissionJob; import org.apache.flink.client.testjar.FailingJob; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DeploymentOptions; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.PipelineOptionsInternal; +import org.apache.flink.core.testutils.FlinkAssertions; +import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; import org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherRunnerFactory; @@ -38,7 +41,10 @@ import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.executiongraph.ErrorInfo; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServicesWithLeadershipControl; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; @@ -48,11 +54,13 @@ import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; import org.apache.flink.runtime.rest.JobRestEndpointFactory; import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.TestLogger; +import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import java.time.Duration; import java.util.UUID; @@ -61,11 +69,10 @@ import java.util.function.Supplier; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; /** Integration tests related to {@link ApplicationDispatcherBootstrap}. */ -public class ApplicationDispatcherBootstrapITCase extends TestLogger { +@ExtendWith(TestLoggerExtension.class) +public class ApplicationDispatcherBootstrapITCase { private static final Duration TIMEOUT = Duration.ofMinutes(10); @@ -127,7 +134,9 @@ public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exc cluster.requestJobResult(ApplicationDispatcherBootstrap.ZERO_JOB_ID); haServices.revokeDispatcherLeadership(); // make sure the leadership is revoked to avoid race conditions - assertEquals(ApplicationStatus.UNKNOWN, firstJobResult.get().getApplicationStatus()); + assertThat(firstJobResult.get()) + .extracting(JobResult::getApplicationStatus) + .isEqualTo(ApplicationStatus.UNKNOWN); haServices.grantDispatcherLeadership(); // job is suspended, wait until it's running @@ -141,10 +150,11 @@ public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exc BlockingJob.unblock(blockId); // and wait for it to actually finish - final CompletableFuture secondJobResult = - cluster.requestJobResult(ApplicationDispatcherBootstrap.ZERO_JOB_ID); - assertTrue(secondJobResult.get().isSuccess()); - assertEquals(ApplicationStatus.SUCCEEDED, secondJobResult.get().getApplicationStatus()); + final JobResult secondJobResult = + cluster.requestJobResult(ApplicationDispatcherBootstrap.ZERO_JOB_ID).get(); + assertThat(secondJobResult.isSuccess()).isTrue(); + assertThat(secondJobResult.getApplicationStatus()) + .isEqualTo(ApplicationStatus.SUCCEEDED); // the cluster should shut down automatically once the application completes awaitClusterStopped(cluster, deadline); @@ -153,6 +163,64 @@ public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exc } } + @Test + public void testDirtyJobResultRecoveryInApplicationMode() throws Exception { + final Deadline deadline = Deadline.fromNow(TIMEOUT); + final Configuration configuration = new Configuration(); + configuration.set(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name()); + configuration.set(DeploymentOptions.TARGET, EmbeddedExecutor.NAME); + configuration.set(ClientOptions.CLIENT_RETRY_PERIOD, Duration.ofMillis(100)); + final TestingMiniClusterConfiguration clusterConfiguration = + TestingMiniClusterConfiguration.newBuilder() + .setConfiguration(configuration) + .build(); + + // having a dirty entry in the JobResultStore should make the ApplicationDispatcherBootstrap + // implementation fail to submit the job + final JobResultStore jobResultStore = new EmbeddedJobResultStore(); + jobResultStore.createDirtyResult( + new JobResultEntry( + TestingJobResultStore.createSuccessfulJobResult( + ApplicationDispatcherBootstrap.ZERO_JOB_ID))); + final EmbeddedHaServicesWithLeadershipControl haServices = + new EmbeddedHaServicesWithLeadershipControl(TestingUtils.defaultExecutor()) { + + @Override + public JobResultStore getJobResultStore() { + return jobResultStore; + } + }; + + final TestingMiniCluster.Builder clusterBuilder = + TestingMiniCluster.newBuilder(clusterConfiguration) + .setHighAvailabilityServicesSupplier(() -> haServices) + .setDispatcherResourceManagerComponentFactorySupplier( + createApplicationModeDispatcherResourceManagerComponentFactorySupplier( + clusterConfiguration.getConfiguration(), + ErrorHandlingSubmissionJob.createPackagedProgram())); + try (final MiniCluster cluster = clusterBuilder.build()) { + // start mini cluster and submit the job + cluster.start(); + + // the cluster should shut down automatically once the application completes + awaitClusterStopped(cluster, deadline); + } + + FlinkAssertions.assertThatChainOfCauses(ErrorHandlingSubmissionJob.getSubmissionException()) + .as( + "The job's main method shouldn't have been succeeded due to a DuplicateJobSubmissionException.") + .hasAtLeastOneElementOfType(DuplicateJobSubmissionException.class); + + assertThat( + jobResultStore.hasDirtyJobResultEntry( + ApplicationDispatcherBootstrap.ZERO_JOB_ID)) + .isFalse(); + assertThat( + jobResultStore.hasCleanJobResultEntry( + ApplicationDispatcherBootstrap.ZERO_JOB_ID)) + .isTrue(); + } + @Test public void testSubmitFailedJobOnApplicationError() throws Exception { final Deadline deadline = Deadline.fromNow(TIMEOUT); diff --git a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapTest.java b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapTest.java index f6fd8db2eea21..9d6ae78c5b9e1 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapTest.java @@ -45,7 +45,7 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.SerializedThrowable; -import org.apache.flink.util.TestLogger; +import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.concurrent.ScheduledExecutor; import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; @@ -53,6 +53,7 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -79,7 +80,8 @@ import static org.junit.jupiter.api.Assertions.fail; /** Tests for the {@link ApplicationDispatcherBootstrap}. */ -public class ApplicationDispatcherBootstrapTest extends TestLogger { +@ExtendWith(TestLoggerExtension.class) +public class ApplicationDispatcherBootstrapTest { private static final int TIMEOUT_SECONDS = 10; @@ -641,7 +643,7 @@ public void testDuplicateJobSubmissionWithTerminatedJobId() throws Throwable { * In this scenario, job result is no longer present in the {@link * org.apache.flink.runtime.dispatcher.Dispatcher dispatcher} (job has terminated and job * manager failed over), but we know that job has already terminated from {@link - * org.apache.flink.runtime.highavailability.RunningJobsRegistry running jobs registry}. + * org.apache.flink.runtime.highavailability.JobResultStore}. */ @Test public void testDuplicateJobSubmissionWithTerminatedJobIdWithUnknownResult() throws Throwable { @@ -675,7 +677,7 @@ public void testDuplicateJobSubmissionWithTerminatedJobIdWithUnknownResult() thr * In this scenario, job result is no longer present in the {@link * org.apache.flink.runtime.dispatcher.Dispatcher dispatcher} (job has terminated and job * manager failed over), but we know that job has already terminated from {@link - * org.apache.flink.runtime.highavailability.RunningJobsRegistry running jobs registry}. + * org.apache.flink.runtime.highavailability.JobResultStore}. */ @Test public void testDuplicateJobSubmissionWithTerminatedJobIdWithUnknownResultAttached() diff --git a/flink-clients/src/test/java/org/apache/flink/client/testjar/ErrorHandlingSubmissionJob.java b/flink-clients/src/test/java/org/apache/flink/client/testjar/ErrorHandlingSubmissionJob.java new file mode 100644 index 0000000000000..83492c257eea2 --- /dev/null +++ b/flink-clients/src/test/java/org/apache/flink/client/testjar/ErrorHandlingSubmissionJob.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.client.testjar; + +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.api.java.io.DiscardingOutputFormat; +import org.apache.flink.client.cli.CliFrontendTestUtils; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.client.program.ProgramInvocationException; +import org.apache.flink.util.FlinkException; + +import java.io.File; +import java.io.FileNotFoundException; +import java.net.MalformedURLException; +import java.util.Arrays; +import java.util.Collections; +import java.util.concurrent.atomic.AtomicReference; + +/** + * {@code ErrorHandlingSubmissionJob} provides a factory method for creating a {@link + * PackagedProgram} that monitors the job submission within the job's {@code main} method. + */ +public class ErrorHandlingSubmissionJob { + + private static final AtomicReference SUBMISSION_EXCEPTION = new AtomicReference<>(); + + public static PackagedProgram createPackagedProgram() throws FlinkException { + try { + return PackagedProgram.newBuilder() + .setUserClassPaths( + Collections.singletonList( + new File(CliFrontendTestUtils.getTestJarPath()) + .toURI() + .toURL())) + .setEntryPointClassName(ErrorHandlingSubmissionJob.class.getName()) + .build(); + } catch (ProgramInvocationException | FileNotFoundException | MalformedURLException e) { + throw new FlinkException("Could not load the provided entrypoint class.", e); + } + } + + public static void main(String[] args) throws Exception { + final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); + env.fromCollection(Arrays.asList(1, 2, 3)) + .map(element -> element + 1) + .output(new DiscardingOutputFormat<>()); + + try { + env.execute(); + } catch (Exception e) { + SUBMISSION_EXCEPTION.set(e); + throw e; + } + } + + public static Exception getSubmissionException() { + return SUBMISSION_EXCEPTION.get(); + } +} diff --git a/flink-connectors/flink-sql-connector-aws-kinesis-data-streams/pom.xml b/flink-connectors/flink-sql-connector-aws-kinesis-data-streams/pom.xml new file mode 100644 index 0000000000000..0e146c91df14a --- /dev/null +++ b/flink-connectors/flink-sql-connector-aws-kinesis-data-streams/pom.xml @@ -0,0 +1,118 @@ + + + + + + flink-connectors + org.apache.flink + 1.15-SNAPSHOT + .. + + 4.0.0 + + flink-sql-connector-aws-kinesis-data-streams + Flink : Connectors : SQL : AWS Kinesis Data Streams + + + + org.apache.flink + flink-connector-aws-kinesis-data-streams + ${project.version} + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + org.apache.flink:flink-table-common + org.apache.flink:flink-streaming-java + org.apache.flink:flink-connector-base + org.apache.flink:flink-connector-aws-base + org.apache.flink:flink-connector-aws-kinesis-data-streams + software.amazon.awssdk:* + org.reactivestreams:* + com.typesafe.netty:* + org.apache.httpcomponents:* + io.netty:* + + + + + software.amazon + org.apache.flink.kinesis-streams.shaded.software.amazon + + + org.reactivestreams + org.apache.flink.kinesis-streams.shaded.org.reactivestreams + + + com.typesafe.netty + org.apache.flink.kinesis-streams.shaded.com.typesafe.netty + + + org.apache.http + org.apache.flink.kinesis-streams.shaded.org.apache.http + + + io.netty + org.apache.flink.kinesis-streams.shaded.io.netty + + + + + software.amazon.awssdk:* + + META-INF/services/** + + + + org.apache.flink:flink-connector-aws-kinesis-data-streams:* + + profile + + + + org.apache.flink:flink-connector-aws-base:* + + profile + + + + + + + + + + diff --git a/flink-connectors/flink-sql-connector-aws-kinesis-data-streams/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-aws-kinesis-data-streams/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..55ec4cac0f12c --- /dev/null +++ b/flink-connectors/flink-sql-connector-aws-kinesis-data-streams/src/main/resources/META-INF/NOTICE @@ -0,0 +1,49 @@ +flink-sql-connector-aws-kinesis-data-streams + +Copyright 2014-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- software.amazon.awssdk:kinesis:2.17.52 +- software.amazon.awssdk:aws-cbor-protocol:2.17.52 +- software.amazon.awssdk:aws-json-protocol:2.17.52 +- software.amazon.awssdk:protocol-core:2.17.52 +- software.amazon.awssdk:profiles:2.17.52 +- software.amazon.awssdk:sdk-core:2.17.52 +- software.amazon.awssdk:auth:2.17.52 +- software.amazon.eventstream:eventstream:1.0.1 +- software.amazon.awssdk:http-client-spi:2.17.52 +- software.amazon.awssdk:regions:2.17.52 +- software.amazon.awssdk:annotations:2.17.52 +- software.amazon.awssdk:utils:2.17.52 +- software.amazon.awssdk:aws-core:2.17.52 +- software.amazon.awssdk:metrics-spi:2.17.52 +- software.amazon.awssdk:apache-client:2.17.52 +- software.amazon.awssdk:netty-nio-client:2.17.52 +- software.amazon.awssdk:sts:2.17.52 +- software.amazon.awssdk:aws-query-protocol:2.17.52 +- software.amazon.awssdk:json-utils:2.17.52 +- software.amazon.awssdk:third-party-jackson-core:2.17.52 +- software.amazon.awssdk:third-party-jackson-dataformat-cbor:2.17.52 +- io.netty:netty-codec-http:4.1.68.Final +- io.netty:netty-codec-http2:4.1.68.Final +- io.netty:netty-codec:4.1.68.Final +- io.netty:netty-transport:4.1.68.Final +- io.netty:netty-resolver:4.1.68.Final +- io.netty:netty-common:4.1.68.Final +- io.netty:netty-buffer:4.1.68.Final +- io.netty:netty-handler:4.1.68.Final +- io.netty:netty-transport-native-epoll:linux-x86_64:4.1.68.Final +- io.netty:netty-transport-native-unix-common:4.1.68.Final +- com.typesafe.netty:netty-reactive-streams-http:2.0.5 +- com.typesafe.netty:netty-reactive-streams:2.0.5 +- org.apache.httpcomponents:httpclient:4.5.13 +- org.apache.httpcomponents:httpcore:4.4.14 + + +This project bundles the following dependencies under the Creative Commons Zero license (https://creativecommons.org/publicdomain/zero/1.0/). + +- org.reactivestreams:reactive-streams:1.0.3 diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml index 4e65f8d038791..74954126e0036 100644 --- a/flink-connectors/pom.xml +++ b/flink-connectors/pom.xml @@ -104,6 +104,7 @@ under the License. flink-sql-connector-hive-2.3.6 flink-sql-connector-hive-3.1.2 flink-sql-connector-kafka + flink-sql-connector-aws-kinesis-data-streams flink-sql-connector-kinesis flink-sql-connector-rabbitmq diff --git a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java index b87682a80c242..8ee274ae97c76 100644 --- a/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java +++ b/flink-core/src/main/java/org/apache/flink/util/CollectionUtil.java @@ -29,6 +29,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; import java.util.function.Function; @@ -55,6 +56,10 @@ public static boolean isNullOrEmpty(Map map) { return map == null || map.isEmpty(); } + public static Set ofNullable(@Nullable T obj) { + return obj == null ? Collections.emptySet() : Collections.singleton(obj); + } + public static Stream mapWithIndex( Collection input, final BiFunction mapper) { final AtomicInteger count = new AtomicInteger(0); diff --git a/flink-core/src/test/java/org/apache/flink/util/CollectionUtilTest.java b/flink-core/src/test/java/org/apache/flink/util/CollectionUtilTest.java index 4b069a09d1683..abeec238879b6 100644 --- a/flink-core/src/test/java/org/apache/flink/util/CollectionUtilTest.java +++ b/flink-core/src/test/java/org/apache/flink/util/CollectionUtilTest.java @@ -18,14 +18,17 @@ package org.apache.flink.util; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collection; import java.util.List; +import static org.assertj.core.api.Assertions.assertThat; + /** Tests for java collection utilities. */ +@ExtendWith(TestLoggerExtension.class) public class CollectionUtilTest { @Test @@ -33,12 +36,20 @@ public void testPartition() { List list = Arrays.asList(1, 2, 3, 4); Collection> partitioned = CollectionUtil.partition(list, 4); - Assert.assertEquals( - "List partitioned into the an incorrect number of partitions", - 4, - partitioned.size()); - for (List partition : partitioned) { - Assert.assertEquals("Unexpected number of elements in partition", 1, partition.size()); - } + assertThat(partitioned) + .as("List partitioned into the an incorrect number of partitions") + .hasSize(4); + assertThat(partitioned).allSatisfy(partition -> assertThat(partition).hasSize(1)); + } + + @Test + public void testOfNullableWithNull() { + assertThat(CollectionUtil.ofNullable(null)).isEmpty(); + } + + @Test + public void testFromNullableWithObject() { + final Object element = new Object(); + assertThat(CollectionUtil.ofNullable(element)).singleElement().isEqualTo(element); } } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/pom.xml new file mode 100644 index 0000000000000..1818c62beb9fb --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/pom.xml @@ -0,0 +1,117 @@ + + + + + + flink-end-to-end-tests + org.apache.flink + 1.15-SNAPSHOT + .. + + 4.0.0 + + flink-end-to-end-tests-kinesis-streams + Flink : E2E Tests : Kinesis SQL tests + jar + + + + org.apache.flink + flink-streaming-java + ${project.version} + test + + + + org.apache.flink + flink-end-to-end-tests-common + ${project.version} + test + + + + org.apache.flink + flink-connector-aws-kinesis-data-streams + ${project.version} + test + test-jar + + + com.typesafe.netty + netty-reactive-streams-http + + + + + com.fasterxml.jackson.core + jackson-databind + test + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy + pre-integration-test + + copy + + + + + + + org.apache.flink + flink-sql-connector-aws-kinesis-data-streams + ${project.version} + sql-kinesis-streams.jar + jar + ${project.build.directory}/dependencies + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.22.1 + + + + + true + true + true + true + + + + + + + + diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/java/org/apache/flink/connector/kinesis/table/test/KinesisDataStreamsTableApiIT.java b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/java/org/apache/flink/connector/kinesis/table/test/KinesisDataStreamsTableApiIT.java new file mode 100644 index 0000000000000..998c5438b0e2a --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/java/org/apache/flink/connector/kinesis/table/test/KinesisDataStreamsTableApiIT.java @@ -0,0 +1,265 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.kinesis.table.test; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.connectors.kinesis.testutils.KinesaliteContainer; +import org.apache.flink.tests.util.TestUtils; +import org.apache.flink.tests.util.flink.SQLJobSubmission; +import org.apache.flink.tests.util.flink.container.FlinkContainers; +import org.apache.flink.util.DockerImageVersions; + +import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.Timeout; +import org.rnorth.ducttape.ratelimits.RateLimiter; +import org.rnorth.ducttape.ratelimits.RateLimiterBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.Network; +import org.testcontainers.utility.DockerImageName; +import software.amazon.awssdk.core.SdkSystemSetting; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.CreateStreamRequest; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamRequest; +import software.amazon.awssdk.services.kinesis.model.GetRecordsRequest; +import software.amazon.awssdk.services.kinesis.model.GetShardIteratorRequest; +import software.amazon.awssdk.services.kinesis.model.Record; +import software.amazon.awssdk.services.kinesis.model.ShardIteratorType; +import software.amazon.awssdk.services.kinesis.model.StreamStatus; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; + +import static java.util.concurrent.TimeUnit.SECONDS; + +/** End-to-end test for Kinesis DataStream Table API Sink using Kinesalite. */ +public class KinesisDataStreamsTableApiIT { + + private static final Logger LOGGER = + LoggerFactory.getLogger(KinesisDataStreamsTableApiIT.class); + + private static final String ORDERS_STREAM = "orders"; + private static final String INTER_CONTAINER_KINESALITE_ALIAS = "kinesalite"; + private static final String DEFAULT_FIRST_SHARD_NAME = "shardId-000000000000"; + + private final Path sqlConnectorKinesisJar = TestUtils.getResource(".*kinesis-streams.jar"); + private static final Network network = Network.newNetwork(); + + @ClassRule public static final Timeout TIMEOUT = new Timeout(10, TimeUnit.MINUTES); + + @ClassRule + public static final KinesaliteContainer KINESALITE = + new KinesaliteContainer(DockerImageName.parse(DockerImageVersions.KINESALITE)) + .withNetwork(network) + .withNetworkAliases(INTER_CONTAINER_KINESALITE_ALIAS); + + public static final FlinkContainers FLINK = + FlinkContainers.builder() + .setEnvironmentVariable("AWS_CBOR_DISABLE", "1") + .setEnvironmentVariable( + "FLINK_ENV_JAVA_OPTS", + "-Dorg.apache.flink.kinesis-streams.shaded.com.amazonaws.sdk.disableCertChecking -Daws.cborEnabled=false") + .setNetwork(network) + .setLogger(LOGGER) + .dependsOn(KINESALITE) + .build(); + + @BeforeClass + public static void setupFlink() throws Exception { + FLINK.start(); + } + + @AfterClass + public static void stopFlink() { + FLINK.stop(); + } + + @Before + public void setUp() throws Exception { + prepareStream(ORDERS_STREAM); + System.setProperty(SdkSystemSetting.CBOR_ENABLED.property(), "false"); + } + + @After + public void teardown() { + System.clearProperty(SdkSystemSetting.CBOR_ENABLED.property()); + } + + @Test + public void testTableApiSourceAndSink() throws Exception { + executeSqlStatements(readSqlFile("send-orders.sql")); + List expected = + ImmutableList.of( + new Order("A", 10), + new Order("B", 12), + new Order("C", 14), + new Order("D", 16), + new Order("E", 18)); + // result order is not guaranteed + List result = readAllOrdersFromKinesis(); + Assertions.assertThat(result).containsAll(expected); + } + + private void prepareStream(String streamName) throws Exception { + final RateLimiter rateLimiter = + RateLimiterBuilder.newBuilder() + .withRate(1, SECONDS) + .withConstantThroughput() + .build(); + + KinesisAsyncClient kinesisClient = KINESALITE.getHostClient(); + kinesisClient + .createStream( + CreateStreamRequest.builder().streamName(streamName).shardCount(1).build()) + .get(); + + Deadline deadline = Deadline.fromNow(Duration.ofMinutes(1)); + while (!rateLimiter.getWhenReady(() -> streamExists(streamName))) { + if (deadline.isOverdue()) { + throw new RuntimeException("Failed to create stream within time"); + } + } + } + + private boolean streamExists(final String streamName) { + try { + return KINESALITE + .getHostClient() + .describeStream( + DescribeStreamRequest.builder().streamName(streamName).build()) + .get() + .streamDescription() + .streamStatus() + == StreamStatus.ACTIVE; + } catch (Exception e) { + return false; + } + } + + private List readAllOrdersFromKinesis() throws Exception { + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(10)); + List orders; + do { + orders = + readMessagesFromStream( + recordBytes -> fromJson(new String(recordBytes), Order.class)); + + } while (deadline.hasTimeLeft() && orders.size() < 5); + + return orders; + } + + private void executeSqlStatements(final List sqlLines) throws Exception { + FLINK.submitSQLJob( + new SQLJobSubmission.SQLJobSubmissionBuilder(sqlLines) + .addJars(sqlConnectorKinesisJar) + .build()); + } + + private List readSqlFile(final String resourceName) throws Exception { + return Files.readAllLines(Paths.get(getClass().getResource("/" + resourceName).toURI())); + } + + private T fromJson(final String json, final Class type) { + try { + return new ObjectMapper().readValue(json, type); + } catch (JsonProcessingException e) { + throw new RuntimeException("Test Failure.", e); + } + } + + private List readMessagesFromStream(Function deserialiser) throws Exception { + String shardIterator = + KINESALITE + .getHostClient() + .getShardIterator( + GetShardIteratorRequest.builder() + .shardId(DEFAULT_FIRST_SHARD_NAME) + .shardIteratorType(ShardIteratorType.TRIM_HORIZON) + .streamName(KinesisDataStreamsTableApiIT.ORDERS_STREAM) + .build()) + .get() + .shardIterator(); + + List records = + KINESALITE + .getHostClient() + .getRecords( + GetRecordsRequest.builder().shardIterator(shardIterator).build()) + .get() + .records(); + List messages = new ArrayList<>(); + records.forEach(record -> messages.add(deserialiser.apply(record.data().asByteArray()))); + return messages; + } + + /** POJO class for orders used by e2e test. */ + public static class Order { + private final String code; + private final int quantity; + + public Order( + @JsonProperty("code") final String code, @JsonProperty("quantity") int quantity) { + this.code = code; + this.quantity = quantity; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + Order order = (Order) o; + return quantity == order.quantity && Objects.equals(code, order.code); + } + + @Override + public int hashCode() { + return Objects.hash(code, quantity); + } + + @Override + public String toString() { + return String.format("Order{code: %s, quantity: %d}", code, quantity); + } + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/resources/log4j2-test.properties b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/resources/log4j2-test.properties new file mode 100644 index 0000000000000..e463a0e1a088c --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/resources/log4j2-test.properties @@ -0,0 +1,28 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = INFO +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/resources/send-orders.sql b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/resources/send-orders.sql new file mode 100644 index 0000000000000..29f2c885ed3cc --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-kinesis-streams/src/test/resources/send-orders.sql @@ -0,0 +1,36 @@ +--/* +-- * Licensed to the Apache Software Foundation (ASF) under one +-- * or more contributor license agreements. See the NOTICE file +-- * distributed with this work for additional information +-- * regarding copyright ownership. The ASF licenses this file +-- * to you under the Apache License, Version 2.0 (the +-- * "License"); you may not use this file except in compliance +-- * with the License. You may obtain a copy of the License at +-- * +-- * http://www.apache.org/licenses/LICENSE-2.0 +-- * +-- * Unless required by applicable law or agreed to in writing, software +-- * distributed under the License is distributed on an "AS IS" BASIS, +-- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- * See the License for the specific language governing permissions and +-- * limitations under the License. +-- */ + +CREATE TABLE orders ( + `code` STRING, + `quantity` BIGINT +) WITH ( + 'connector' = 'kinesis', + 'stream' = 'orders', + 'aws.region' = 'us-east-1', + 'aws.endpoint' = 'https://kinesalite:4567', + 'aws.credentials.provider' = 'BASIC', + 'aws.credentials.basic.accesskeyid' = 'access key', + 'aws.credentials.basic.secretkey' ='secret key', + 'aws.trust.all.certificates' = 'true', + 'sink.http-client.protocol.version' = 'HTTP1_1', + 'sink.batch.max-size' = '1', + 'format' = 'json' +); + +INSERT INTO orders VALUES ('A', 10),('B', 12),('C', 14),('D', 16),('E', 18); diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index aaf3b94151957..5ad33b8913118 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -80,6 +80,7 @@ under the License. flink-glue-schema-registry-avro-test flink-glue-schema-registry-json-test flink-end-to-end-tests-scala + flink-end-to-end-tests-kinesis-streams diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvBulkWriter.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvBulkWriter.java new file mode 100644 index 0000000000000..69b3f9bce283f --- /dev/null +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvBulkWriter.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.csv; + +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.core.fs.FSDataOutputStream; +import org.apache.flink.formats.common.Converter; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectWriter; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema; + +import javax.annotation.Nullable; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** A simple {@link BulkWriter} implementation based on Jackson CSV transformations. */ +class CsvBulkWriter implements BulkWriter { + + private final FSDataOutputStream stream; + private final Converter converter; + @Nullable private final C converterContext; + private final ObjectWriter csvWriter; + + CsvBulkWriter( + CsvMapper mapper, + CsvSchema schema, + Converter converter, + @Nullable C converterContext, + FSDataOutputStream stream) { + checkNotNull(mapper); + checkNotNull(schema); + + this.converter = checkNotNull(converter); + this.stream = checkNotNull(stream); + this.converterContext = converterContext; + this.csvWriter = mapper.writer(schema); + + // Prevent Jackson's writeValue() method calls from closing the stream. + mapper.getFactory().disable(JsonGenerator.Feature.AUTO_CLOSE_TARGET); + } + + /** + * Builds a writer with Jackson schema and a type converter. + * + * @param mapper The specialized mapper for producing CSV. + * @param schema The schema that defined the mapping properties. + * @param converter The type converter that converts incoming elements of type {@code } into + * elements of type JsonNode. + * @param stream The output stream. + * @param The type of the elements accepted by this writer. + * @param The type of the converter context. + * @param The type of the elements produced by this writer. + */ + static CsvBulkWriter forSchema( + CsvMapper mapper, + CsvSchema schema, + Converter converter, + @Nullable C converterContext, + FSDataOutputStream stream) { + return new CsvBulkWriter<>(mapper, schema, converter, converterContext, stream); + } + + /** + * Builds a writer based on a POJO class definition. + * + * @param pojoClass The class of the POJO. + * @param stream The output stream. + * @param The type of the elements accepted by this writer. + */ + static CsvBulkWriter forPojo(Class pojoClass, FSDataOutputStream stream) { + final Converter converter = (value, context) -> value; + final CsvMapper csvMapper = new CsvMapper(); + final CsvSchema schema = csvMapper.schemaFor(pojoClass).withoutQuoteChar(); + return new CsvBulkWriter<>(csvMapper, schema, converter, null, stream); + } + + @Override + public void addElement(T element) throws IOException { + final R r = converter.convert(element, converterContext); + csvWriter.writeValue(stream, r); + } + + @Override + public void flush() throws IOException { + stream.flush(); + } + + @Override + public void finish() throws IOException { + stream.sync(); + } +} diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvCommons.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvCommons.java new file mode 100644 index 0000000000000..48825748f8251 --- /dev/null +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvCommons.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.csv; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; + +import org.apache.commons.lang3.StringEscapeUtils; + +import java.util.HashSet; +import java.util.Set; + +import static org.apache.flink.formats.csv.CsvFormatOptions.ALLOW_COMMENTS; +import static org.apache.flink.formats.csv.CsvFormatOptions.ARRAY_ELEMENT_DELIMITER; +import static org.apache.flink.formats.csv.CsvFormatOptions.DISABLE_QUOTE_CHARACTER; +import static org.apache.flink.formats.csv.CsvFormatOptions.ESCAPE_CHARACTER; +import static org.apache.flink.formats.csv.CsvFormatOptions.FIELD_DELIMITER; +import static org.apache.flink.formats.csv.CsvFormatOptions.IGNORE_PARSE_ERRORS; +import static org.apache.flink.formats.csv.CsvFormatOptions.NULL_LITERAL; +import static org.apache.flink.formats.csv.CsvFormatOptions.QUOTE_CHARACTER; + +/** A class with common CSV format constants and utility methods. */ +class CsvCommons { + + public static final String IDENTIFIER = "csv"; + + // ------------------------------------------------------------------------ + // Validation + // ------------------------------------------------------------------------ + + static void validateFormatOptions(ReadableConfig tableOptions) { + final boolean hasQuoteCharacter = tableOptions.getOptional(QUOTE_CHARACTER).isPresent(); + final boolean isDisabledQuoteCharacter = tableOptions.get(DISABLE_QUOTE_CHARACTER); + if (isDisabledQuoteCharacter && hasQuoteCharacter) { + throw new ValidationException( + "Format cannot define a quote character and disabled quote character at the same time."); + } + // Validate the option value must be a single char. + validateCharacterVal(tableOptions, FIELD_DELIMITER, true); + validateCharacterVal(tableOptions, ARRAY_ELEMENT_DELIMITER); + validateCharacterVal(tableOptions, QUOTE_CHARACTER); + validateCharacterVal(tableOptions, ESCAPE_CHARACTER); + } + + /** Validates the option {@code option} value must be a Character. */ + private static void validateCharacterVal( + ReadableConfig tableOptions, ConfigOption option) { + validateCharacterVal(tableOptions, option, false); + } + + /** + * Validates the option {@code option} value must be a Character. + * + * @param tableOptions the table options + * @param option the config option + * @param unescape whether to unescape the option value + */ + private static void validateCharacterVal( + ReadableConfig tableOptions, ConfigOption option, boolean unescape) { + if (!tableOptions.getOptional(option).isPresent()) { + return; + } + + final String value = + unescape + ? StringEscapeUtils.unescapeJava(tableOptions.get(option)) + : tableOptions.get(option); + if (value.length() != 1) { + throw new ValidationException( + String.format( + "Option '%s.%s' must be a string with single character, but was: %s", + IDENTIFIER, option.key(), tableOptions.get(option))); + } + } + + public static Set> optionalOptions() { + Set> options = new HashSet<>(); + options.add(FIELD_DELIMITER); + options.add(DISABLE_QUOTE_CHARACTER); + options.add(QUOTE_CHARACTER); + options.add(ALLOW_COMMENTS); + options.add(IGNORE_PARSE_ERRORS); + options.add(ARRAY_ELEMENT_DELIMITER); + options.add(ESCAPE_CHARACTER); + options.add(NULL_LITERAL); + return options; + } + + public static Set> forwardOptions() { + Set> options = new HashSet<>(); + options.add(FIELD_DELIMITER); + options.add(DISABLE_QUOTE_CHARACTER); + options.add(QUOTE_CHARACTER); + options.add(ALLOW_COMMENTS); + options.add(ARRAY_ELEMENT_DELIMITER); + options.add(ESCAPE_CHARACTER); + options.add(NULL_LITERAL); + return options; + } +} diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java new file mode 100644 index 0000000000000..8ea2280eaf1ef --- /dev/null +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileFormatFactory.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.csv; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.api.common.serialization.BulkWriter.Factory; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.file.src.FileSourceSplit; +import org.apache.flink.connector.file.src.impl.StreamFormatAdapter; +import org.apache.flink.connector.file.src.reader.BulkFormat; +import org.apache.flink.connector.file.table.factories.BulkReaderFormatFactory; +import org.apache.flink.connector.file.table.factories.BulkWriterFormatFactory; +import org.apache.flink.connector.file.table.format.BulkDecodingFormat; +import org.apache.flink.formats.common.Converter; +import org.apache.flink.formats.csv.RowDataToCsvConverters.RowDataToCsvConverter; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.format.ProjectableDecodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource.Context; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema; + +import org.apache.commons.lang3.StringEscapeUtils; + +import java.util.Collections; +import java.util.Set; + +import static org.apache.flink.formats.csv.CsvFormatOptions.ALLOW_COMMENTS; +import static org.apache.flink.formats.csv.CsvFormatOptions.ARRAY_ELEMENT_DELIMITER; +import static org.apache.flink.formats.csv.CsvFormatOptions.DISABLE_QUOTE_CHARACTER; +import static org.apache.flink.formats.csv.CsvFormatOptions.ESCAPE_CHARACTER; +import static org.apache.flink.formats.csv.CsvFormatOptions.FIELD_DELIMITER; +import static org.apache.flink.formats.csv.CsvFormatOptions.IGNORE_PARSE_ERRORS; +import static org.apache.flink.formats.csv.CsvFormatOptions.NULL_LITERAL; +import static org.apache.flink.formats.csv.CsvFormatOptions.QUOTE_CHARACTER; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** CSV format factory for file system. */ +@Internal +public class CsvFileFormatFactory implements BulkReaderFormatFactory, BulkWriterFormatFactory { + + @Override + public String factoryIdentifier() { + return CsvCommons.IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + return Collections.emptySet(); + } + + @Override + public Set> optionalOptions() { + return CsvCommons.optionalOptions(); + } + + @Override + public Set> forwardOptions() { + return CsvCommons.forwardOptions(); + } + + @Override + public BulkDecodingFormat createDecodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + + return new CsvBulkDecodingFormat(formatOptions); + } + + private static class CsvBulkDecodingFormat + implements BulkDecodingFormat, + ProjectableDecodingFormat> { + + private final ReadableConfig formatOptions; + + public CsvBulkDecodingFormat(ReadableConfig formatOptions) { + checkNotNull(formatOptions); + this.formatOptions = formatOptions; + } + + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) + public BulkFormat createRuntimeDecoder( + Context context, DataType physicalDataType, int[][] projections) { + + final DataType projectedDataType = Projection.of(projections).project(physicalDataType); + final RowType projectedRowType = (RowType) projectedDataType.getLogicalType(); + + final RowType physicalRowType = (RowType) physicalDataType.getLogicalType(); + final CsvSchema schema = buildCsvSchema(physicalRowType, formatOptions); + + final boolean ignoreParseErrors = + formatOptions.getOptional(IGNORE_PARSE_ERRORS).isPresent(); + final Converter converter = + (Converter) + new CsvToRowDataConverters(ignoreParseErrors) + .createRowConverter(projectedRowType, true); + CsvReaderFormat csvReaderFormat = + new CsvReaderFormat<>( + new CsvMapper(), + schema, + JsonNode.class, + converter, + context.createTypeInformation(projectedDataType), + ignoreParseErrors); + return new StreamFormatAdapter<>(csvReaderFormat); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + } + + @Override + public EncodingFormat> createEncodingFormat( + DynamicTableFactory.Context context, ReadableConfig formatOptions) { + return new EncodingFormat>() { + @Override + public BulkWriter.Factory createRuntimeEncoder( + DynamicTableSink.Context context, DataType physicalDataType) { + + final RowType rowType = (RowType) physicalDataType.getLogicalType(); + final CsvSchema schema = buildCsvSchema(rowType, formatOptions); + + final RowDataToCsvConverter converter = + RowDataToCsvConverters.createRowConverter(rowType); + + final CsvMapper mapper = new CsvMapper(); + final ObjectNode container = mapper.createObjectNode(); + + final RowDataToCsvConverter.RowDataToCsvFormatConverterContext converterContext = + new RowDataToCsvConverter.RowDataToCsvFormatConverterContext( + mapper, container); + + return out -> + CsvBulkWriter.forSchema(mapper, schema, converter, converterContext, out); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + }; + } + + private static CsvSchema buildCsvSchema(RowType rowType, ReadableConfig options) { + final CsvSchema csvSchema = CsvRowSchemaConverter.convert(rowType); + final CsvSchema.Builder csvBuilder = csvSchema.rebuild(); + // format properties + options.getOptional(FIELD_DELIMITER) + .map(s -> StringEscapeUtils.unescapeJava(s).charAt(0)) + .ifPresent(csvBuilder::setColumnSeparator); + + if (options.get(DISABLE_QUOTE_CHARACTER)) { + csvBuilder.disableQuoteChar(); + } else { + options.getOptional(QUOTE_CHARACTER) + .map(s -> s.charAt(0)) + .ifPresent(csvBuilder::setQuoteChar); + } + + options.getOptional(ALLOW_COMMENTS).ifPresent(csvBuilder::setAllowComments); + + options.getOptional(ARRAY_ELEMENT_DELIMITER) + .ifPresent(csvBuilder::setArrayElementSeparator); + + options.getOptional(ESCAPE_CHARACTER) + .map(s -> s.charAt(0)) + .ifPresent(csvBuilder::setEscapeChar); + + options.getOptional(NULL_LITERAL).ifPresent(csvBuilder::setNullValue); + + return csvBuilder.build(); + } +} diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileSystemFormatFactory.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileSystemFormatFactory.java deleted file mode 100644 index 74b2c215fe59f..0000000000000 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFileSystemFormatFactory.java +++ /dev/null @@ -1,282 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.formats.csv; - -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.connector.file.table.factories.FileSystemFormatFactory; -import org.apache.flink.core.fs.FileInputSplit; -import org.apache.flink.core.fs.Path; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.utils.PartitionPathUtils; - -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.MappingIterator; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema; - -import org.apache.commons.lang3.StringEscapeUtils; - -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStreamReader; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.Set; -import java.util.stream.Collectors; - -import static org.apache.flink.formats.csv.CsvFormatFactory.validateFormatOptions; -import static org.apache.flink.formats.csv.CsvFormatOptions.ALLOW_COMMENTS; -import static org.apache.flink.formats.csv.CsvFormatOptions.ARRAY_ELEMENT_DELIMITER; -import static org.apache.flink.formats.csv.CsvFormatOptions.DISABLE_QUOTE_CHARACTER; -import static org.apache.flink.formats.csv.CsvFormatOptions.ESCAPE_CHARACTER; -import static org.apache.flink.formats.csv.CsvFormatOptions.FIELD_DELIMITER; -import static org.apache.flink.formats.csv.CsvFormatOptions.IGNORE_PARSE_ERRORS; -import static org.apache.flink.formats.csv.CsvFormatOptions.NULL_LITERAL; -import static org.apache.flink.formats.csv.CsvFormatOptions.QUOTE_CHARACTER; - -/** CSV format factory for file system. */ -public class CsvFileSystemFormatFactory implements FileSystemFormatFactory { - - public static final String IDENTIFIER = "csv"; - - @Override - public String factoryIdentifier() { - return IDENTIFIER; - } - - @Override - public Set> requiredOptions() { - return new HashSet<>(); - } - - @Override - public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(FIELD_DELIMITER); - options.add(DISABLE_QUOTE_CHARACTER); - options.add(QUOTE_CHARACTER); - options.add(ALLOW_COMMENTS); - options.add(IGNORE_PARSE_ERRORS); - options.add(ARRAY_ELEMENT_DELIMITER); - options.add(ESCAPE_CHARACTER); - options.add(NULL_LITERAL); - return options; - } - - @Override - public InputFormat createReader(ReaderContext context) { - ReadableConfig options = context.getFormatOptions(); - validateFormatOptions(options); - - RowType formatRowType = context.getFormatRowType(); - - String[] fieldNames = context.getSchema().getFieldNames(); - List projectFields = - Arrays.stream(context.getProjectFields()) - .mapToObj(idx -> fieldNames[idx]) - .collect(Collectors.toList()); - List csvFields = - Arrays.stream(fieldNames) - .filter(field -> !context.getPartitionKeys().contains(field)) - .collect(Collectors.toList()); - - int[] csvSelectFieldToProjectFieldMapping = - context.getFormatProjectFields().stream() - .mapToInt(projectFields::indexOf) - .toArray(); - int[] csvSelectFieldToCsvFieldMapping = - context.getFormatProjectFields().stream().mapToInt(csvFields::indexOf).toArray(); - - CsvSchema csvSchema = buildCsvSchema(formatRowType, options); - - boolean ignoreParseErrors = options.get(IGNORE_PARSE_ERRORS); - - return new CsvInputFormat( - context.getPaths(), - context.getSchema().getFieldDataTypes(), - context.getSchema().getFieldNames(), - csvSchema, - formatRowType, - context.getProjectFields(), - context.getPartitionKeys(), - context.getDefaultPartName(), - context.getPushedDownLimit(), - csvSelectFieldToProjectFieldMapping, - csvSelectFieldToCsvFieldMapping, - ignoreParseErrors); - } - - private CsvSchema buildCsvSchema(RowType rowType, ReadableConfig options) { - CsvSchema csvSchema = CsvRowSchemaConverter.convert(rowType); - CsvSchema.Builder csvBuilder = csvSchema.rebuild(); - // format properties - options.getOptional(FIELD_DELIMITER) - .map(s -> StringEscapeUtils.unescapeJava(s).charAt(0)) - .ifPresent(csvBuilder::setColumnSeparator); - - options.getOptional(QUOTE_CHARACTER) - .map(s -> s.charAt(0)) - .ifPresent(csvBuilder::setQuoteChar); - - options.getOptional(ALLOW_COMMENTS).ifPresent(csvBuilder::setAllowComments); - - options.getOptional(ARRAY_ELEMENT_DELIMITER) - .ifPresent(csvBuilder::setArrayElementSeparator); - - options.getOptional(ESCAPE_CHARACTER) - .map(s -> s.charAt(0)) - .ifPresent(csvBuilder::setEscapeChar); - - options.getOptional(NULL_LITERAL).ifPresent(csvBuilder::setNullValue); - - return csvBuilder.build(); - } - - /** InputFormat that reads csv record into {@link RowData}. */ - public static class CsvInputFormat extends AbstractCsvInputFormat { - private static final long serialVersionUID = 1L; - - private final RowType formatRowType; - private final DataType[] fieldTypes; - private final String[] fieldNames; - private final int[] selectFields; - private final List partitionKeys; - private final String defaultPartValue; - private final long limit; - private final int[] csvSelectFieldToProjectFieldMapping; - private final int[] csvSelectFieldToCsvFieldMapping; - private final boolean ignoreParseErrors; - - private transient InputStreamReader inputStreamReader; - private transient BufferedReader reader; - private transient boolean end; - private transient long emitted; - // reuse object for per record - private transient GenericRowData rowData; - private transient CsvToRowDataConverters.CsvToRowDataConverter runtimeConverter; - private transient MappingIterator iterator; - - public CsvInputFormat( - Path[] filePaths, - DataType[] fieldTypes, - String[] fieldNames, - CsvSchema csvSchema, - RowType formatRowType, - int[] selectFields, - List partitionKeys, - String defaultPartValue, - long limit, - int[] csvSelectFieldToProjectFieldMapping, - int[] csvSelectFieldToCsvFieldMapping, - boolean ignoreParseErrors) { - super(filePaths, csvSchema); - this.fieldTypes = fieldTypes; - this.fieldNames = fieldNames; - this.formatRowType = formatRowType; - this.partitionKeys = partitionKeys; - this.defaultPartValue = defaultPartValue; - this.selectFields = selectFields; - this.limit = limit; - this.emitted = 0; - this.csvSelectFieldToProjectFieldMapping = csvSelectFieldToProjectFieldMapping; - this.csvSelectFieldToCsvFieldMapping = csvSelectFieldToCsvFieldMapping; - this.ignoreParseErrors = ignoreParseErrors; - } - - @Override - public void open(FileInputSplit split) throws IOException { - super.open(split); - this.end = false; - this.inputStreamReader = new InputStreamReader(csvInputStream); - this.reader = new BufferedReader(inputStreamReader); - this.rowData = - PartitionPathUtils.fillPartitionValueForRecord( - fieldNames, - fieldTypes, - selectFields, - partitionKeys, - currentSplit.getPath(), - defaultPartValue); - this.iterator = - new CsvMapper() - .readerFor(JsonNode.class) - .with(csvSchema) - .readValues(csvInputStream); - prepareRuntimeConverter(); - } - - private void prepareRuntimeConverter() { - this.runtimeConverter = - new CsvToRowDataConverters(ignoreParseErrors) - .createRowConverter(formatRowType, true); - } - - @Override - public boolean reachedEnd() throws IOException { - return emitted >= limit || end; - } - - @Override - public RowData nextRecord(RowData reuse) throws IOException { - GenericRowData csvRow = null; - while (csvRow == null) { - try { - JsonNode root = iterator.nextValue(); - csvRow = (GenericRowData) runtimeConverter.convert(root); - } catch (NoSuchElementException e) { - end = true; - return null; - } catch (Throwable t) { - if (!ignoreParseErrors) { - throw new IOException("Failed to deserialize CSV row.", t); - } - } - } - - GenericRowData returnRecord = rowData; - for (int i = 0; i < csvSelectFieldToCsvFieldMapping.length; i++) { - returnRecord.setField( - csvSelectFieldToProjectFieldMapping[i], - csvRow.getField(csvSelectFieldToCsvFieldMapping[i])); - } - emitted++; - return returnRecord; - } - - @Override - public void close() throws IOException { - super.close(); - if (reader != null) { - reader.close(); - reader = null; - } - if (inputStreamReader != null) { - inputStreamReader.close(); - inputStreamReader = null; - } - } - } -} diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFormatFactory.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFormatFactory.java index ddfd685b0d227..0bd8aa6433199 100644 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFormatFactory.java +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvFormatFactory.java @@ -24,7 +24,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; @@ -41,7 +40,6 @@ import org.apache.commons.lang3.StringEscapeUtils; import java.util.Collections; -import java.util.HashSet; import java.util.Set; import static org.apache.flink.formats.csv.CsvFormatOptions.ALLOW_COMMENTS; @@ -61,13 +59,11 @@ public final class CsvFormatFactory implements DeserializationFormatFactory, SerializationFormatFactory { - public static final String IDENTIFIER = "csv"; - @Override public DecodingFormat> createDecodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); - validateFormatOptions(formatOptions); + CsvCommons.validateFormatOptions(formatOptions); return new DecodingFormat>() { @Override @@ -93,7 +89,7 @@ public ChangelogMode getChangelogMode() { public EncodingFormat> createEncodingFormat( DynamicTableFactory.Context context, ReadableConfig formatOptions) { FactoryUtil.validateFactoryOptions(this, formatOptions); - validateFormatOptions(formatOptions); + CsvCommons.validateFormatOptions(formatOptions); return new EncodingFormat>() { @Override @@ -115,7 +111,7 @@ public ChangelogMode getChangelogMode() { @Override public String factoryIdentifier() { - return IDENTIFIER; + return CsvCommons.IDENTIFIER; } @Override @@ -125,76 +121,12 @@ public Set> requiredOptions() { @Override public Set> optionalOptions() { - Set> options = new HashSet<>(); - options.add(FIELD_DELIMITER); - options.add(DISABLE_QUOTE_CHARACTER); - options.add(QUOTE_CHARACTER); - options.add(ALLOW_COMMENTS); - options.add(IGNORE_PARSE_ERRORS); - options.add(ARRAY_ELEMENT_DELIMITER); - options.add(ESCAPE_CHARACTER); - options.add(NULL_LITERAL); - return options; + return CsvCommons.optionalOptions(); } @Override public Set> forwardOptions() { - Set> options = new HashSet<>(); - options.add(FIELD_DELIMITER); - options.add(DISABLE_QUOTE_CHARACTER); - options.add(QUOTE_CHARACTER); - options.add(ALLOW_COMMENTS); - options.add(ARRAY_ELEMENT_DELIMITER); - options.add(ESCAPE_CHARACTER); - options.add(NULL_LITERAL); - return options; - } - - // ------------------------------------------------------------------------ - // Validation - // ------------------------------------------------------------------------ - - static void validateFormatOptions(ReadableConfig tableOptions) { - final boolean hasQuoteCharacter = tableOptions.getOptional(QUOTE_CHARACTER).isPresent(); - final boolean isDisabledQuoteCharacter = tableOptions.get(DISABLE_QUOTE_CHARACTER); - if (isDisabledQuoteCharacter && hasQuoteCharacter) { - throw new ValidationException( - "Format cannot define a quote character and disabled quote character at the same time."); - } - // Validate the option value must be a single char. - validateCharacterVal(tableOptions, FIELD_DELIMITER, true); - validateCharacterVal(tableOptions, ARRAY_ELEMENT_DELIMITER); - validateCharacterVal(tableOptions, QUOTE_CHARACTER); - validateCharacterVal(tableOptions, ESCAPE_CHARACTER); - } - - /** Validates the option {@code option} value must be a Character. */ - private static void validateCharacterVal( - ReadableConfig tableOptions, ConfigOption option) { - validateCharacterVal(tableOptions, option, false); - } - - /** - * Validates the option {@code option} value must be a Character. - * - * @param tableOptions the table options - * @param option the config option - * @param unescape whether to unescape the option value - */ - private static void validateCharacterVal( - ReadableConfig tableOptions, ConfigOption option, boolean unescape) { - if (tableOptions.getOptional(option).isPresent()) { - final String value = - unescape - ? StringEscapeUtils.unescapeJava(tableOptions.get(option)) - : tableOptions.get(option); - if (value.length() != 1) { - throw new ValidationException( - String.format( - "Option '%s.%s' must be a string with single character, but was: %s", - IDENTIFIER, option.key(), tableOptions.get(option))); - } - } + return CsvCommons.forwardOptions(); } // ------------------------------------------------------------------------ diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvReaderFormat.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvReaderFormat.java new file mode 100644 index 0000000000000..1ce00fd4d70d8 --- /dev/null +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvReaderFormat.java @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.csv; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.file.src.reader.SimpleStreamFormat; +import org.apache.flink.connector.file.src.reader.StreamFormat; +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.formats.common.Converter; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.MappingIterator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema; + +import javax.annotation.Nullable; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@code StreamFormat} for reading CSV files. + * + *

The following example shows how to create a {@code CsvReaderFormat} where the schema for CSV + * parsing is automatically derived based on the fields of a POJO class. + * + *

{@code
+ * CsvReaderFormat csvFormat = CsvReaderFormat.forPojo(SomePojo.class);
+ * FileSource source =
+ *         FileSource.forRecordStreamFormat(csvFormat, Path.fromLocalFile(filesPath)).build();
+ * }
+ * + * Note: you might need to add {@code @JsonPropertyOrder({field1, field2, ...})} annotation from + * the {@code Jackson} library to your class definition with the fields order exactly matching those + * of the CSV file columns). + * + *

If you need more fine-grained control over the CSV schema or the parsing options, use the more + * low-level {@code forSchema} static factory method based on the {@code Jackson} library utilities: + * + *

{@code
+ * CsvMapper mapper = new CsvMapper();
+ * CsvSchema schema = mapper.schemaFor(SomePojo.class)
+ *                          .withColumnSeparator('|');
+ * CsvReaderFormat csvFormat =
+ *          CsvReaderFormat.forSchema(mapper,schema, TypeInformation.of(SomePojo.class));
+ * FileSource source =
+ *         FileSource.forRecordStreamFormat(csvFormat, Path.fromLocalFile(filesPath)).build();
+ * }
+ * + * @param The type of the returned elements. + */ +@PublicEvolving +public class CsvReaderFormat extends SimpleStreamFormat { + + private static final long serialVersionUID = 1L; + + private final CsvMapper mapper; + private final CsvSchema schema; + private final Class rootType; + private final Converter converter; + private final TypeInformation typeInformation; + private boolean ignoreParseErrors; + + @SuppressWarnings("unchecked") + CsvReaderFormat( + CsvMapper mapper, + CsvSchema schema, + Class rootType, + Converter converter, + TypeInformation typeInformation, + boolean ignoreParseErrors) { + this.mapper = checkNotNull(mapper); + this.schema = checkNotNull(schema); + this.rootType = (Class) checkNotNull(rootType); + this.typeInformation = checkNotNull(typeInformation); + this.converter = (Converter) checkNotNull(converter); + this.ignoreParseErrors = ignoreParseErrors; + } + + /** + * Builds a new {@code CsvReaderFormat} using a {@code CsvSchema}. + * + * @param schema The Jackson CSV schema configured for parsing specific CSV files. + * @param typeInformation The Flink type descriptor of the returned elements. + * @param The type of the returned elements. + */ + public static CsvReaderFormat forSchema( + CsvSchema schema, TypeInformation typeInformation) { + return forSchema(new CsvMapper(), schema, typeInformation); + } + + /** + * Builds a new {@code CsvReaderFormat} using a {@code CsvSchema} and a pre-created {@code + * CsvMapper}. + * + * @param mapper The pre-created {@code CsvMapper}. + * @param schema The Jackson CSV schema configured for parsing specific CSV files. + * @param typeInformation The Flink type descriptor of the returned elements. + * @param The type of the returned elements. + */ + public static CsvReaderFormat forSchema( + CsvMapper mapper, CsvSchema schema, TypeInformation typeInformation) { + return new CsvReaderFormat<>( + mapper, + schema, + typeInformation.getTypeClass(), + (value, context) -> value, + typeInformation, + false); + } + + /** + * Builds a new {@code CsvReaderFormat} for reading CSV files mapped to the provided POJO class + * definition. Produced reader uses default mapper and schema settings, use {@code forSchema} if + * you need customizations. + * + * @param pojoType The type class of the POJO. + * @param The type of the returned elements. + */ + public static CsvReaderFormat forPojo(Class pojoType) { + CsvMapper mapper = new CsvMapper(); + return forSchema( + mapper, + mapper.schemaFor(pojoType).withoutQuoteChar(), + TypeInformation.of(pojoType)); + } + + /** + * Returns a new {@code CsvReaderFormat} configured to ignore all parsing errors. All thye other + * options directly carried over from the subject of the method call. + */ + public CsvReaderFormat withIgnoreParseErrors() { + return new CsvReaderFormat( + this.mapper, + this.schema, + this.rootType, + this.converter, + this.typeInformation, + true); + } + + @Override + public StreamFormat.Reader createReader(Configuration config, FSDataInputStream stream) + throws IOException { + return new Reader<>( + mapper.readerFor(rootType).with(schema).readValues(stream), + converter, + ignoreParseErrors); + } + + @Override + public TypeInformation getProducedType() { + return typeInformation; + } + + // ------------------------------------------------------------------------ + + /** The actual reader for the {@code CsvFormat}. */ + private static final class Reader implements StreamFormat.Reader { + private final MappingIterator iterator; + private final Converter converter; + private final boolean ignoreParseErrors; + + public Reader( + MappingIterator iterator, + Converter converter, + boolean ignoreParseErrors) { + this.iterator = checkNotNull(iterator); + this.converter = checkNotNull(converter); + this.ignoreParseErrors = ignoreParseErrors; + } + + @Nullable + @Override + public T read() throws IOException { + while (iterator.hasNext()) { + try { + R nextElement = iterator.next(); + return converter.convert(nextElement, null); + } catch (Throwable t) { + if (!ignoreParseErrors) { + throw t; + } + } + } + return null; + } + + @Override + public void close() throws IOException { + iterator.close(); + } + } +} diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataSerializationSchema.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataSerializationSchema.java index 87abdbaaebbe4..16d7c966a52b6 100644 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataSerializationSchema.java +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataSerializationSchema.java @@ -65,6 +65,11 @@ public final class CsvRowDataSerializationSchema implements SerializationSchema< /** Reusable object node. */ private transient ObjectNode root; + /** Reusable converter context. */ + private transient RowDataToCsvConverters.RowDataToCsvConverter + .RowDataToCsvFormatConverterContext + converterContext; + private CsvRowDataSerializationSchema(RowType rowType, CsvSchema csvSchema) { this.rowType = rowType; this.runtimeConverter = RowDataToCsvConverters.createRowConverter(rowType); @@ -132,9 +137,12 @@ public CsvRowDataSerializationSchema build() { public byte[] serialize(RowData row) { if (root == null) { root = csvMapper.createObjectNode(); + converterContext = + new RowDataToCsvConverters.RowDataToCsvConverter + .RowDataToCsvFormatConverterContext(csvMapper, root); } try { - runtimeConverter.convert(csvMapper, root, row); + runtimeConverter.convert(row, converterContext); return objectWriter.writeValueAsBytes(root); } catch (Throwable t) { throw new RuntimeException(String.format("Could not serialize row '%s'.", row), t); diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvToRowDataConverters.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvToRowDataConverters.java index 526ed5ca96bff..ed307a9a07fb8 100644 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvToRowDataConverters.java +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvToRowDataConverters.java @@ -19,6 +19,7 @@ package org.apache.flink.formats.csv; import org.apache.flink.annotation.Internal; +import org.apache.flink.formats.common.Converter; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericArrayData; import org.apache.flink.table.data.GenericRowData; @@ -65,7 +66,12 @@ public CsvToRowDataConverters(boolean ignoreParseErrors) { * data structures. */ @FunctionalInterface - public interface CsvToRowDataConverter extends Serializable { + interface CsvToRowDataConverter extends Converter { + @Override + default Object convert(JsonNode source, Void context) { + return convert(source); + } + Object convert(JsonNode jsonNode); } @@ -305,7 +311,7 @@ private CsvToRowDataConverter createArrayConverter(ArrayType arrayType) { } private static void validateArity(int expected, int actual, boolean ignoreParseErrors) { - if (expected != actual && !ignoreParseErrors) { + if (expected > actual && !ignoreParseErrors) { throw new RuntimeException( "Row length mismatch. " + expected diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/RowDataToCsvConverters.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/RowDataToCsvConverters.java index ea1bd8d626877..693e89198e2cb 100644 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/RowDataToCsvConverters.java +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/RowDataToCsvConverters.java @@ -19,6 +19,7 @@ package org.apache.flink.formats.csv; import org.apache.flink.annotation.Internal; +import org.apache.flink.formats.common.Converter; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.RowData; @@ -57,8 +58,23 @@ public class RowDataToCsvConverters implements Serializable { * Runtime converter that converts objects of Flink Table & SQL internal data structures to * corresponding {@link JsonNode}s. */ - public interface RowDataToCsvConverter extends Serializable { - JsonNode convert(CsvMapper csvMapper, ContainerNode container, RowData row); + interface RowDataToCsvConverter + extends Converter< + RowData, JsonNode, RowDataToCsvConverter.RowDataToCsvFormatConverterContext> { + /** + * Converter context for passing the {@code CsvMapper} and the {@code container} that can be + * reused between transformations of the individual elements for performance reasons. + */ + class RowDataToCsvFormatConverterContext implements Serializable { + CsvMapper csvMapper; + ContainerNode container; + + public RowDataToCsvFormatConverterContext( + CsvMapper csvMapper, ContainerNode container) { + this.csvMapper = csvMapper; + this.container = container; + } + } } private interface RowFieldConverter extends Serializable { @@ -80,14 +96,15 @@ public static RowDataToCsvConverter createRowConverter(RowType type) { .map(RowDataToCsvConverters::createNullableRowFieldConverter) .toArray(RowFieldConverter[]::new); final int rowArity = type.getFieldCount(); - return (csvMapper, container, row) -> { + return (row, context) -> { // top level reuses the object node container - final ObjectNode objectNode = (ObjectNode) container; + final ObjectNode objectNode = (ObjectNode) context.container; for (int i = 0; i < rowArity; i++) { try { objectNode.set( fieldNames[i], - fieldConverters[i].convert(csvMapper, container, row, i)); + fieldConverters[i].convert( + context.csvMapper, context.container, row, i)); } catch (Throwable t) { throw new RuntimeException( String.format("Fail to serialize at field: %s.", fieldNames[i]), t); diff --git a/flink-formats/flink-csv/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats/flink-csv/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index fdd66f47f9448..0e962ecdd2ecc 100644 --- a/flink-formats/flink-csv/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-formats/flink-csv/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -13,5 +13,5 @@ # See the License for the specific language governing permissions and # limitations under the License. +org.apache.flink.formats.csv.CsvFileFormatFactory org.apache.flink.formats.csv.CsvFormatFactory -org.apache.flink.formats.csv.CsvFileSystemFormatFactory diff --git a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFilesystemBatchITCase.java b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFilesystemBatchITCase.java index f6dadc59fbbb9..4b9436998ac72 100644 --- a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFilesystemBatchITCase.java +++ b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFilesystemBatchITCase.java @@ -80,7 +80,22 @@ public void testParseError() throws Exception { new File(path).mkdirs(); File file = new File(path, "test_file"); file.createNewFile(); - FileUtils.writeFileUtf8(file, "x5,5,1,1\n" + "x5,5,2,2,2\n" + "x5,5,1,1"); + FileUtils.writeFileUtf8( + file, "x5,5,1,1\n" + "x5,5,2,2,2\n" + "x5,5,3,3,3,3\n" + "x5,5,1,1"); + + check( + "select * from nonPartitionedTable", + Arrays.asList(Row.of("x5", 5, 1, 1), Row.of("x5", 5, 1, 1))); + } + + @Test + public void testParseErrorLast() throws Exception { + String path = new URI(resultPath()).getPath(); + new File(path).mkdirs(); + File file = new File(path, "test_file"); + file.createNewFile(); + FileUtils.writeFileUtf8( + file, "x5,5,1,1\n" + "x5,5,2,2,2\n" + "x5,5,1,1\n" + "x5,5,3,3,3,3\n"); check( "select * from nonPartitionedTable", diff --git a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFormatFactoryTest.java b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFormatFactoryTest.java index 9c0d4c602b258..54eac4bbf7141 100644 --- a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFormatFactoryTest.java +++ b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/CsvFormatFactoryTest.java @@ -249,7 +249,7 @@ private static Map getAllOptions() { options.put("target", "MyTarget"); options.put("buffer-size", "1000"); - options.put("format", CsvFormatFactory.IDENTIFIER); + options.put("format", CsvCommons.IDENTIFIER); options.put("csv.field-delimiter", ";"); options.put("csv.quote-character", "'"); options.put("csv.allow-comments", "true"); diff --git a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java new file mode 100644 index 0000000000000..a9a64e71d35c4 --- /dev/null +++ b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/DataStreamCsvITCase.java @@ -0,0 +1,411 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.csv; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.serialization.BulkWriter; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.file.sink.FileSink; +import org.apache.flink.connector.file.src.FileSource; +import org.apache.flink.connector.file.src.reader.StreamFormat; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.testutils.AllCallbackWrapper; +import org.apache.flink.formats.common.Converter; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.MiniClusterExtension; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamUtils; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.filesystem.bucketassigners.BasePathBucketAssigner; +import org.apache.flink.streaming.api.operators.collect.ClientAndIterator; +import org.apache.flink.util.TestLoggerExtension; +import org.apache.flink.util.function.FunctionWithException; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonPropertyOrder; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema; + +import org.apache.commons.io.FileUtils; +import org.jetbrains.annotations.NotNull; +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.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.io.PrintWriter; +import java.io.Serializable; +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.UUID; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; + +/** MiniCluster-based integration tests CSV data format. */ +@ExtendWith({TestLoggerExtension.class}) +public class DataStreamCsvITCase { + + private static final int PARALLELISM = 4; + + @TempDir File outDir; + + private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build()); + + @RegisterExtension + private static AllCallbackWrapper allCallbackWrapper = + new AllCallbackWrapper<>(MINI_CLUSTER_RESOURCE); + + // ------------------------------------------------------------------------ + // test data + // ------------------------------------------------------------------------ + + private static final String[] CSV_LINES = + new String[] { + "Berlin,52.5167,13.3833,Germany,DE,Berlin,primary,3644826", + "San Francisco,37.7562,-122.443,United States,US,California,,3592294", + "Beijing,39.905,116.3914,China,CN,Beijing,primary,19433000" + }; + + private static final String[] CSV_LINES_PIPE_SEPARATED = + new String[] { + "Berlin|52.5167|13.3833|Germany|DE|Berlin|primary|3644826", + "San Francisco|37.7562|-122.443|United States|US|California||3592294", + "Beijing|39.905|116.3914|China|CN|Beijing|primary|19433000" + }; + + private static final String[] CSV_LINES_MALFORMED = + new String[] { + "Berlin,52.5167,13.3833,Germany,DE,Berlin,primary,3644826", + "San Francisco,MALFORMED,3592294", + "Beijing,39.905,116.3914,China,CN,Beijing,primary,19433000" + }; + + static final CityPojo[] POJOS = + new CityPojo[] { + new CityPojo( + "Berlin", + new BigDecimal("52.5167"), + new BigDecimal("13.3833"), + "Germany", + "DE", + "Berlin", + "primary", + 3644826L), + new CityPojo( + "San Francisco", + new BigDecimal("37.7562"), + new BigDecimal("-122.443"), + "United States", + "US", + "California", + "", + 3592294L), + new CityPojo( + "Beijing", + new BigDecimal("39.905"), + new BigDecimal("116.3914"), + "China", + "CN", + "Beijing", + "primary", + 19433000L) + }; + + // ------------------------------------------------------------------------ + // test cases + // ------------------------------------------------------------------------ + @Test + public void testCsvReaderFormatFromPojo() throws Exception { + writeFile(outDir, "data.csv", CSV_LINES); + + final CsvReaderFormat csvFormat = CsvReaderFormat.forPojo(CityPojo.class); + final List result = initializeSourceAndReadData(outDir, csvFormat); + + assertThat(Arrays.asList(POJOS)).isEqualTo(result); + } + + @Test + public void testCsvReaderFormatFromSchema() throws Exception { + writeFile(outDir, "data.csv", CSV_LINES_PIPE_SEPARATED); + + CsvMapper mapper = new CsvMapper(); + CsvSchema schema = + mapper.schemaFor(CityPojo.class).withoutQuoteChar().withColumnSeparator('|'); + + final CsvReaderFormat csvFormat = + CsvReaderFormat.forSchema(mapper, schema, TypeInformation.of(CityPojo.class)); + final List result = initializeSourceAndReadData(outDir, csvFormat); + + assertThat(Arrays.asList(POJOS)).isEqualTo(result); + } + + @Test + public void testCsvReaderFormatMalformed() throws Exception { + writeFile(outDir, "data.csv", CSV_LINES_MALFORMED); + + final CsvReaderFormat csvFormat = + CsvReaderFormat.forPojo(CityPojo.class).withIgnoreParseErrors(); + final List result = initializeSourceAndReadData(outDir, csvFormat); + + List expected = new ArrayList<>(); + expected.add(POJOS[0]); + expected.add(POJOS[2]); + + assertThat(expected).isEqualTo(result); + } + + @Test + public void testCustomBulkWriter() throws Exception { + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + + // fromCollection is not bounded, using fromSequence instead + final List pojosList = Arrays.asList(POJOS); // needs to be Serializable + final DataStream sequence = + env.fromSequence(0, POJOS.length - 1).map(Long::intValue); + final DataStream stream = sequence.map(pojosList::get).returns(CityPojo.class); + + FileSink sink = + FileSink.forBulkFormat(new Path(outDir.toURI()), factoryForPojo(CityPojo.class)) + .withBucketAssigner(new BasePathBucketAssigner<>()) + .build(); + + stream.sinkTo(sink); + env.execute(); + + String[] result = getResultsFromSinkFiles(outDir); + + assertThat(result).containsExactlyInAnyOrder(CSV_LINES); + } + + @NotNull + private String[] getResultsFromSinkFiles(File outDir) throws IOException { + final Map contents = getFileContentByPath(outDir); + + List resultList = + contents.entrySet().stream() + .flatMap(e -> Arrays.stream(e.getValue().split("\n"))) + .collect(Collectors.toList()); + + String[] result = resultList.toArray(new String[0]); + return result; + } + + private static BulkWriter.Factory factoryForPojo(Class pojoClass) { + final Converter converter = (value, context) -> value; + final CsvMapper csvMapper = new CsvMapper(); + final CsvSchema schema = csvMapper.schemaFor(pojoClass).withoutQuoteChar(); + return (out) -> new CsvBulkWriter<>(csvMapper, schema, converter, null, out); + } + + private static Map getFileContentByPath(File directory) throws IOException { + Map contents = new HashMap<>(4); + + final Collection filesInBucket = FileUtils.listFiles(directory, null, true); + for (File file : filesInBucket) { + contents.put(file, FileUtils.readFileToString(file)); + } + return contents; + } + + @JsonPropertyOrder({ + "city", + "lat", + "lng", + "country", + "iso2", + "adminName", + "capital", + "population" + }) + static class CityPojo implements Serializable { + public String city; + public BigDecimal lat; + public BigDecimal lng; + public String country; + public String iso2; + public String adminName; + public String capital; + public long population; + + public CityPojo() {} + + public CityPojo( + String city, + BigDecimal lat, + BigDecimal lng, + String country, + String iso2, + String adminName, + String capital, + long population) { + this.city = city; + this.lat = lat; + this.lng = lng; + this.country = country; + this.iso2 = iso2; + this.adminName = adminName; + this.capital = capital; + this.population = population; + } + + @Override + public String toString() { + return "CitiesPojo{" + + "city='" + + city + + '\'' + + ", lat=" + + lat + + ", lng=" + + lng + + ", country='" + + country + + '\'' + + ", iso2='" + + iso2 + + '\'' + + ", adminName='" + + adminName + + '\'' + + ", capital='" + + capital + + '\'' + + ", population=" + + population + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CityPojo that = (CityPojo) o; + return population == that.population + && Objects.equals(city, that.city) + && Objects.equals(lat, that.lat) + && Objects.equals(lng, that.lng) + && Objects.equals(country, that.country) + && Objects.equals(iso2, that.iso2) + && Objects.equals(adminName, that.adminName) + && Objects.equals(capital, that.capital); + } + + @Override + public int hashCode() { + return Objects.hash(city, lat, lng, country, iso2, adminName, capital, population); + } + } + + private static List initializeSourceAndReadData(File testDir, StreamFormat csvFormat) + throws Exception { + final FileSource source = + FileSource.forRecordStreamFormat(csvFormat, Path.fromLocalFile(testDir)).build(); + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); + + final DataStream stream = + env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); + + return getResultsFromStream(stream); + } + + @NotNull + private static List getResultsFromStream(DataStream stream) throws Exception { + final ClientAndIterator client = + DataStreamUtils.collectWithClient(stream, "Bounded Results Fetch"); + + final List result = new ArrayList<>(); + while (client.iterator.hasNext()) { + T next = client.iterator.next(); + result.add(next); + } + return result; + } + + // ------------------------------------------------------------------------ + // Write data utils + // ------------------------------------------------------------------------ + + private static void writeFile(File testDir, String fileName, String[] lines) + throws IOException { + final File file = new File(testDir, fileName); + writeFileAtomically(file, lines); + } + + private static void writeFileAtomically(final File file, final String[] lines) + throws IOException { + writeFileAtomically(file, lines, (v) -> v); + } + + private static void writeFileAtomically( + final File file, + final String[] lines, + final FunctionWithException + streamEncoderFactory) + throws IOException { + + final File stagingFile = + new File(file.getParentFile(), ".tmp-" + UUID.randomUUID().toString()); + + try (final FileOutputStream fileOut = new FileOutputStream(stagingFile); + final OutputStream out = streamEncoderFactory.apply(fileOut); + final OutputStreamWriter encoder = + new OutputStreamWriter(out, StandardCharsets.UTF_8); + final PrintWriter writer = new PrintWriter(encoder)) { + + for (String line : lines) { + writer.println(line); + } + } + + final File parent = file.getParentFile(); + + assertThat(parent.mkdirs() || parent.exists()).isTrue(); + assertThat(stagingFile.renameTo(file)).isTrue(); + } +} diff --git a/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/TableCsvFormatITCase.java b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/TableCsvFormatITCase.java new file mode 100644 index 0000000000000..d51fb9e87c1fb --- /dev/null +++ b/flink-formats/flink-csv/src/test/java/org/apache/flink/formats/csv/TableCsvFormatITCase.java @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.csv; + +import org.apache.flink.formats.common.TimeFormats; +import org.apache.flink.table.planner.runtime.utils.TestData; +import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; +import org.apache.flink.table.planner.utils.JsonPlanTestBase; + +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.utils.DateTimeUtils.toLocalDateTime; + +/** Tests for the CSV file format. */ +public class TableCsvFormatITCase extends JsonPlanTestBase { + + @Test + public void testProjectPushDown() throws Exception { + List data = Arrays.asList("1,1,hi", "2,1,hello", "3,2,hello world"); + createSourceTable("MyTable", data, "a bigint", "b int not null", "c varchar"); + File sinkPath = createSinkTable("MySink", "a bigint", "c varchar"); + + String jsonPlan = tableEnv.getJsonPlan("insert into MySink select a, c from MyTable"); + tableEnv.executeJsonPlan(jsonPlan).await(); + + assertResult(Arrays.asList("1,hi", "2,hello", "3,hello world"), sinkPath); + } + + @Test + public void testReadingMetadata() throws Exception { + createTestValuesSourceTable( + "MyTable", + JavaScalaConversionUtil.toJava(TestData.smallData3()), + new String[] {"a int", "b bigint", "m varchar metadata"}, + new HashMap() { + { + put("readable-metadata", "m:STRING"); + } + }); + + File sinkPath = createSinkTable("MySink", "a bigint", "m varchar"); + + String jsonPlan = tableEnv.getJsonPlan("insert into MySink select a, m from MyTable"); + tableEnv.executeJsonPlan(jsonPlan).await(); + + assertResult(Arrays.asList("1,Hi", "2,Hello", "3,Hello world"), sinkPath); + } + + @Test + public void testFilterPushDown() throws Exception { + List data = Arrays.asList("1,1,hi", "2,1,hello", "3,2,hello world"); + createSourceTable("MyTable", data, "a bigint", "b int not null", "c varchar"); + File sinkPath = createSinkTable("MySink", "a bigint", "b int", "c varchar"); + + String jsonPlan = + tableEnv.getJsonPlan("insert into MySink select * from MyTable where a > 1"); + tableEnv.executeJsonPlan(jsonPlan).await(); + + assertResult(Arrays.asList("2,1,hello", "3,2,hello world"), sinkPath); + } + + @Test + public void testPartitionPushDown() throws Exception { + createTestValuesSourceTable( + "MyTable", + JavaScalaConversionUtil.toJava(TestData.smallData3()), + new String[] {"a int", "p bigint", "c varchar"}, + "p", + new HashMap() { + { + put("partition-list", "p:1;p:2"); + } + }); + File sinkPath = createSinkTable("MySink", "a int", "p bigint", "c varchar"); + + String jsonPlan = + tableEnv.getJsonPlan("insert into MySink select * from MyTable where p = 2"); + tableEnv.executeJsonPlan(jsonPlan).await(); + + assertResult(Arrays.asList("2,2,Hello", "3,2,Hello world"), sinkPath); + } + + @Test + public void testWatermarkPushDown() throws Exception { + createTestValuesSourceTable( + "MyTable", + JavaScalaConversionUtil.toJava(TestData.data3WithTimestamp()), + new String[] { + "a int", + "b bigint", + "c varchar", + "ts timestamp(3)", + "watermark for ts as ts - interval '5' second" + }, + new HashMap() { + { + put("enable-watermark-push-down", "true"); + } + }); + + File sinkPath = createSinkTable("MySink", "a int", "b bigint", "ts timestamp(3)"); + + String jsonPlan = + tableEnv.getJsonPlan("insert into MySink select a, b, ts from MyTable where b = 3"); + tableEnv.executeJsonPlan(jsonPlan).await(); + + assertResult( + Arrays.asList( + "4,3," + formatSqlTimestamp(4000L), + "5,3," + formatSqlTimestamp(5000L), + "6,3," + formatSqlTimestamp(6000L)), + sinkPath); + } + + @Test + public void testPushDowns() throws Exception { + createTestValuesSourceTable( + "MyTable", + JavaScalaConversionUtil.toJava(TestData.data3WithTimestamp()), + new String[] { + "a int", + "b bigint", + "c varchar", + "ts timestamp(3)", + "watermark for ts as ts - interval '5' second" + }, + "b", + new HashMap() { + { + put("readable-metadata", "a:INT"); + put("filterable-fields", "a"); + put("enable-watermark-push-down", "true"); + put("partition-list", "b:1;b:2;b:3;b:4;b:5;b:6"); + } + }); + + File sinkPath = createSinkTable("MySink", "a int", "ts timestamp(3)"); + + String jsonPlan = + tableEnv.getJsonPlan( + "insert into MySink select a, ts from MyTable where b = 3 and a > 4"); + tableEnv.executeJsonPlan(jsonPlan).await(); + + assertResult( + Arrays.asList("5," + formatSqlTimestamp(5000L), "6," + formatSqlTimestamp(6000L)), + sinkPath); + } + + private static String formatSqlTimestamp(long timestamp) { + return TimeFormats.SQL_TIMESTAMP_FORMAT.format(toLocalDateTime(timestamp)); + } + + private void createSourceTable(String tableName, List data, String... fieldNameAndTypes) + throws IOException { + File sourceFile = TEMPORARY_FOLDER.newFile(); + Collections.shuffle(data); + Files.write(sourceFile.toPath(), String.join("\n", data).getBytes()); + + Map properties = new HashMap<>(); + properties.put("connector", "filesystem"); + properties.put("path", sourceFile.getAbsolutePath()); + properties.put("format", "csv"); + + createTestSourceTable(tableName, fieldNameAndTypes, null, properties); + } + + private File createSinkTable(String tableName, String... fieldNameAndTypes) throws IOException { + File sinkPath = TEMPORARY_FOLDER.newFolder(); + + Map properties = new HashMap<>(); + properties.put("connector", "filesystem"); + properties.put("path", sinkPath.getAbsolutePath()); + properties.put("format", "csv"); + properties.put("csv.disable-quote-character", "true"); + + createTestSinkTable(tableName, fieldNameAndTypes, null, properties); + return sinkPath; + } +} diff --git a/flink-formats/flink-csv/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-formats/flink-csv/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 0000000000000..0e962ecdd2ecc --- /dev/null +++ b/flink-formats/flink-csv/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,17 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.flink.formats.csv.CsvFileFormatFactory +org.apache.flink.formats.csv.CsvFormatFactory diff --git a/flink-formats/flink-format-common/src/main/java/org/apache/flink/formats/common/Converter.java b/flink-formats/flink-format-common/src/main/java/org/apache/flink/formats/common/Converter.java new file mode 100644 index 0000000000000..6584227f2e1d2 --- /dev/null +++ b/flink-formats/flink-format-common/src/main/java/org/apache/flink/formats/common/Converter.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.formats.common; + +import org.apache.flink.annotation.PublicEvolving; + +import java.io.Serializable; + +/** + * A generic interface for converting data types. + * + * @param The type of the element to be converted. + * @param The output type. + * @param The context for passing optional conversion instructions. + */ +@PublicEvolving +public interface Converter extends Serializable { + /** + * Converts elements of type {@code From} into elements of type {@code To}. + * + * @param source The element to be converted. + * @param context The context with conversion instructions and utilities. + * @return The element in the target type. + */ + To convert(From source, C context); +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java index 2c318645a46ea..ea78ecbbd2829 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java @@ -80,7 +80,6 @@ private KubernetesCheckpointRecoveryFactory( public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobID, int maxNumberOfCheckpointsToRetain, - ClassLoader userClassLoader, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor) throws Exception { diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesHaServices.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesHaServices.java index e66973f872d6e..7f70e85f81fff 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesHaServices.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesHaServices.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.blob.BlobStoreService; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.highavailability.AbstractHaServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService; import org.apache.flink.runtime.leaderelection.LeaderElectionService; @@ -96,8 +96,8 @@ public class KubernetesHaServices extends AbstractHaServices { Executor executor, Configuration config, BlobStoreService blobStoreService) { + super(config, executor, blobStoreService, new EmbeddedJobResultStore()); - super(config, executor, blobStoreService); this.kubeClient = checkNotNull(kubeClient); this.clusterId = checkNotNull(config.get(KubernetesConfigOptions.CLUSTER_ID)); @@ -145,12 +145,6 @@ public JobGraphStore createJobGraphStore() throws Exception { configuration, kubeClient, getLeaderPathForDispatcher(), lockIdentity); } - @Override - public RunningJobsRegistry createRunningJobsRegistry() { - return new KubernetesRunningJobsRegistry( - kubeClient, getLeaderPathForDispatcher(), lockIdentity); - } - @Override public void internalClose() { configMapSharedWatcher.close(); diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionHaServices.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionHaServices.java index 1a208d25b4a43..d1a313393a355 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionHaServices.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionHaServices.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.blob.BlobStoreService; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.highavailability.AbstractHaServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService; import org.apache.flink.runtime.leaderelection.DefaultMultipleComponentLeaderElectionService; @@ -83,7 +83,7 @@ public class KubernetesMultipleComponentLeaderElectionHaServices extends Abstrac BlobStoreService blobStoreService, FatalErrorHandler fatalErrorHandler) { - super(config, executor, blobStoreService); + super(config, executor, blobStoreService, new EmbeddedJobResultStore()); this.kubeClient = checkNotNull(kubeClient); this.clusterId = checkNotNull(config.get(KubernetesConfigOptions.CLUSTER_ID)); this.fatalErrorHandler = checkNotNull(fatalErrorHandler); @@ -167,11 +167,6 @@ private String getClusterConfigMap() { return clusterId + NAME_SEPARATOR + "cluster-config-map"; } - @Override - protected RunningJobsRegistry createRunningJobsRegistry() { - return new KubernetesRunningJobsRegistry(kubeClient, getClusterConfigMap(), lockIdentity); - } - @Override public void internalClose() throws Exception { Exception exception = null; diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesRunningJobsRegistry.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesRunningJobsRegistry.java deleted file mode 100644 index 7f75c3bbbc245..0000000000000 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesRunningJobsRegistry.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.kubernetes.highavailability; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; -import org.apache.flink.kubernetes.kubeclient.resources.KubernetesConfigMap; -import org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; -import org.apache.flink.util.StringUtils; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Optional; - -import static org.apache.flink.kubernetes.utils.Constants.RUNNING_JOBS_REGISTRY_KEY_PREFIX; -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * {@link RunningJobsRegistry} implementation for Kubernetes. All the running jobs will be stored in - * Dispatcher-leader ConfigMap. The key is the job id with prefix {@link - * org.apache.flink.kubernetes.utils.Constants#RUNNING_JOBS_REGISTRY_KEY_PREFIX}, and value is job - * status. - */ -public class KubernetesRunningJobsRegistry implements RunningJobsRegistry { - - private static final Logger LOG = LoggerFactory.getLogger(KubernetesRunningJobsRegistry.class); - - private final FlinkKubeClient kubeClient; - - private final String configMapName; - - private final String lockIdentity; - - public KubernetesRunningJobsRegistry( - FlinkKubeClient kubeClient, String configMapName, String lockIdentity) { - this.kubeClient = checkNotNull(kubeClient); - this.configMapName = checkNotNull(configMapName); - this.lockIdentity = checkNotNull(lockIdentity); - } - - @Override - public void setJobRunning(JobID jobID) throws IOException { - checkNotNull(jobID); - - writeJobStatusToConfigMap(jobID, JobSchedulingStatus.RUNNING); - } - - @Override - public void setJobFinished(JobID jobID) throws IOException { - checkNotNull(jobID); - - writeJobStatusToConfigMap(jobID, JobSchedulingStatus.DONE); - } - - @Override - public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) throws IOException { - checkNotNull(jobID); - - return kubeClient - .getConfigMap(configMapName) - .map( - configMap -> - getJobStatus(configMap, jobID).orElse(JobSchedulingStatus.PENDING)) - .orElseThrow( - () -> new IOException("ConfigMap " + configMapName + " does not exist.")); - } - - @Override - public void clearJob(JobID jobID) throws IOException { - checkNotNull(jobID); - - try { - kubeClient - .checkAndUpdateConfigMap( - configMapName, - configMap -> { - if (KubernetesLeaderElector.hasLeadership( - configMap, lockIdentity)) { - if (configMap.getData().remove(getKeyForJobId(jobID)) != null) { - return Optional.of(configMap); - } - } - return Optional.empty(); - }) - .get(); - } catch (Exception e) { - throw new IOException( - "Failed to clear job state in ConfigMap " + configMapName + " for job " + jobID, - e); - } - } - - private void writeJobStatusToConfigMap(JobID jobID, JobSchedulingStatus status) - throws IOException { - LOG.debug("Setting scheduling state for job {} to {}.", jobID, status); - final String key = getKeyForJobId(jobID); - try { - kubeClient - .checkAndUpdateConfigMap( - configMapName, - configMap -> { - if (KubernetesLeaderElector.hasLeadership( - configMap, lockIdentity)) { - final Optional optional = - getJobStatus(configMap, jobID); - if (!optional.isPresent() || optional.get() != status) { - configMap.getData().put(key, status.name()); - return Optional.of(configMap); - } - } - return Optional.empty(); - }) - .get(); - } catch (Exception e) { - throw new IOException( - "Failed to set " - + status.name() - + " state in ConfigMap " - + configMapName - + " for job " - + jobID, - e); - } - } - - private Optional getJobStatus(KubernetesConfigMap configMap, JobID jobId) { - final String key = getKeyForJobId(jobId); - final String status = configMap.getData().get(key); - if (!StringUtils.isNullOrWhitespaceOnly(status)) { - return Optional.of(JobSchedulingStatus.valueOf(status)); - } - return Optional.empty(); - } - - private String getKeyForJobId(JobID jobId) { - return RUNNING_JOBS_REGISTRY_KEY_PREFIX + jobId.toString(); - } -} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java index 14fdc2860c938..6e0f25e48540f 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/Constants.java @@ -96,7 +96,6 @@ public class Constants { public static final String LEADER_SESSION_ID_KEY = "sessionId"; public static final String JOB_GRAPH_STORE_KEY_PREFIX = "jobGraph-"; public static final String SUBMITTED_JOBGRAPH_FILE_PREFIX = "submittedJobGraph"; - public static final String RUNNING_JOBS_REGISTRY_KEY_PREFIX = "runningJobsRegistry-"; public static final String CHECKPOINT_COUNTER_KEY = "counter"; public static final String CHECKPOINT_ID_KEY_PREFIX = "checkpointID-"; public static final String COMPLETED_CHECKPOINT_FILE_SUFFIX = "completedCheckpoint"; diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java index e02d3a824e2f7..36982c7efe8a7 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java @@ -33,8 +33,8 @@ import org.apache.flink.kubernetes.kubeclient.resources.KubernetesPod; import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStoreUtils; import org.apache.flink.runtime.checkpoint.DefaultCompletedCheckpointStore; -import org.apache.flink.runtime.checkpoint.DefaultCompletedCheckpointStoreUtils; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.DefaultJobGraphStore; @@ -323,7 +323,7 @@ public static CompletedCheckpointStore createCompletedCheckpointStore( k -> k.startsWith(CHECKPOINT_ID_KEY_PREFIX), lockIdentity); Collection checkpoints = - DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( + CompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( stateHandleStore, KubernetesCheckpointStoreUtil.INSTANCE); return new DefaultCompletedCheckpointStore<>( diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesRunningJobsRegistryTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesRunningJobsRegistryTest.java deleted file mode 100644 index d77c572df478f..0000000000000 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesRunningJobsRegistryTest.java +++ /dev/null @@ -1,123 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.kubernetes.highavailability; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.core.testutils.FlinkMatchers; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; - -import org.junit.Test; - -import java.io.IOException; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.fail; - -/** Tests for {@link KubernetesRunningJobsRegistry} operations. */ -public class KubernetesRunningJobsRegistryTest extends KubernetesHighAvailabilityTestBase { - - private final JobID jobID = JobID.generate(); - - @Test - public void testSetAndGetJobStatus() throws Exception { - new Context() { - { - runTest( - () -> { - leaderCallbackGrantLeadership(); - - final KubernetesRunningJobsRegistry runningJobsRegistry = - new KubernetesRunningJobsRegistry( - flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); - runningJobsRegistry.setJobRunning(jobID); - assertThat( - runningJobsRegistry.getJobSchedulingStatus(jobID), - is(RunningJobsRegistry.JobSchedulingStatus.RUNNING)); - }); - } - }; - } - - @Test - public void testGetJobStatusNonExisting() throws Exception { - new Context() { - { - runTest( - () -> { - leaderCallbackGrantLeadership(); - - final KubernetesRunningJobsRegistry runningJobsRegistry = - new KubernetesRunningJobsRegistry( - flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); - final JobID jobId = JobID.generate(); - assertThat( - runningJobsRegistry.getJobSchedulingStatus(jobId), - is(RunningJobsRegistry.JobSchedulingStatus.PENDING)); - }); - } - }; - } - - @Test - public void testGetJobStatusConfigMapNotExist() throws Exception { - new Context() { - { - runTest( - () -> { - final KubernetesRunningJobsRegistry runningJobsRegistry = - new KubernetesRunningJobsRegistry( - flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); - try { - runningJobsRegistry.getJobSchedulingStatus(JobID.generate()); - fail("Exception should be thrown."); - } catch (IOException ex) { - final String msg = - "ConfigMap " + LEADER_CONFIGMAP_NAME + " does not exist"; - assertThat(ex, FlinkMatchers.containsMessage(msg)); - } - }); - } - }; - } - - @Test - public void testClearJob() throws Exception { - new Context() { - { - runTest( - () -> { - leaderCallbackGrantLeadership(); - - final KubernetesRunningJobsRegistry runningJobsRegistry = - new KubernetesRunningJobsRegistry( - flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); - runningJobsRegistry.setJobFinished(jobID); - assertThat( - runningJobsRegistry.getJobSchedulingStatus(jobID), - is(RunningJobsRegistry.JobSchedulingStatus.DONE)); - runningJobsRegistry.clearJob(jobID); - assertThat( - runningJobsRegistry.getJobSchedulingStatus(jobID), - is(RunningJobsRegistry.JobSchedulingStatus.PENDING)); - }); - } - }; - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java index 24be572862e2d..0a5628f1de897 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java @@ -25,9 +25,12 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.SecurityOptions; +import org.apache.flink.runtime.dispatcher.cleanup.GloballyCleanableResource; +import org.apache.flink.runtime.dispatcher.cleanup.LocallyCleanableResource; import org.apache.flink.runtime.net.SSLUtils; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FileUtils; +import org.apache.flink.util.FlinkException; import org.apache.flink.util.NetUtils; import org.apache.flink.util.Reference; import org.apache.flink.util.ShutdownHookUtil; @@ -74,7 +77,12 @@ * the directory structure to store the BLOBs or temporarily cache them. */ public class BlobServer extends Thread - implements BlobService, BlobWriter, PermanentBlobService, TransientBlobService { + implements BlobService, + BlobWriter, + PermanentBlobService, + TransientBlobService, + LocallyCleanableResource, + GloballyCleanableResource { /** The log object used for debugging. */ private static final Logger LOG = LoggerFactory.getLogger(BlobServer.class); @@ -861,15 +869,14 @@ public boolean deletePermanent(JobID jobId, PermanentBlobKey key) { } /** - * Removes all BLOBs from local and HA store belonging to the given job ID. + * Deletes locally stored artifacts for the job represented by the given {@link JobID}. This + * doesn't touch the job's entry in the {@link BlobStore} to enable recovering. * - * @param jobId ID of the job this blob belongs to - * @param cleanupBlobStoreFiles True if the corresponding blob store files shall be cleaned up - * as well. Otherwise false. - * @return true if the job directory is successfully deleted or non-existing; - * false otherwise + * @param jobId The {@code JobID} of the job that is subject to cleanup. + * @throws IOException if the cleanup failed. */ - public boolean cleanupJob(JobID jobId, boolean cleanupBlobStoreFiles) { + @Override + public void localCleanup(JobID jobId) throws IOException { checkNotNull(jobId); final File jobDir = @@ -877,44 +884,58 @@ public boolean cleanupJob(JobID jobId, boolean cleanupBlobStoreFiles) { BlobUtils.getStorageLocationPath( storageDir.deref().getAbsolutePath(), jobId)); + FileUtils.deleteDirectory(jobDir); + + // NOTE on why blobExpiryTimes are not cleaned up: + // Instead of going through blobExpiryTimes, keep lingering entries - they + // will be cleaned up by the timer task which tolerates non-existing files + // If inserted again with the same IDs (via put()), the TTL will be updated + // again. + } + + /** + * Removes all BLOBs from local and HA store belonging to the given {@link JobID}. + * + * @param jobId ID of the job this blob belongs to + * @throws Exception if the cleanup fails. + */ + @Override + public void globalCleanup(JobID jobId) throws Exception { + checkNotNull(jobId); + readWriteLock.writeLock().lock(); try { - // delete locally - boolean deletedLocally = false; - try { - FileUtils.deleteDirectory(jobDir); - - // NOTE: Instead of going through blobExpiryTimes, keep lingering entries - they - // will be cleaned up by the timer task which tolerates non-existing files - // If inserted again with the same IDs (via put()), the TTL will be updated - // again. + Exception exception = null; - deletedLocally = true; + try { + localCleanup(jobId); } catch (IOException e) { - LOG.warn( - "Failed to locally delete BLOB storage directory at " - + jobDir.getAbsolutePath(), - e); + exception = e; } - // delete in HA blob store files - final boolean deletedHA = !cleanupBlobStoreFiles || blobStore.deleteAll(jobId); + if (!blobStore.deleteAll(jobId)) { + exception = + ExceptionUtils.firstOrSuppressed( + new FlinkException( + "Error while cleaning up the BlobStore for job " + jobId), + exception); + } - return deletedLocally && deletedHA; + ExceptionUtils.tryRethrowException(exception); } finally { readWriteLock.writeLock().unlock(); } } - public void retainJobs(Collection jobsToRetain) throws IOException { + public void retainJobs(Collection jobsToRetain) throws Exception { if (storageDir.deref().exists()) { final Set jobsToRemove = BlobUtils.listExistingJobs(storageDir.deref().toPath()); jobsToRemove.removeAll(jobsToRetain); for (JobID jobToRemove : jobsToRemove) { - cleanupJob(jobToRemove, true); + globalCleanup(jobToRemove); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java index d03cc745d0687..e8b53cc9170b4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/FileSystemBlobStore.java @@ -158,7 +158,13 @@ private boolean delete(String blobPath) { Path path = new Path(blobPath); - boolean result = fileSystem.delete(path, true); + boolean result = true; + if (fileSystem.exists(path)) { + result = fileSystem.delete(path, true); + } else { + LOG.debug( + "The given path {} is not present anymore. No deletion is required.", path); + } // send a call to delete the directory containing the file. This will // fail (and be ignored) when some files still exist. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java index 81d70097471a4..ab0b5ef8506ed 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java @@ -34,7 +34,6 @@ public interface CheckpointRecoveryFactory { * * @param jobId Job ID to recover checkpoints for * @param maxNumberOfCheckpointsToRetain Maximum number of checkpoints to retain - * @param userClassLoader User code class loader of the job * @param sharedStateRegistryFactory Simple factory to produce {@link SharedStateRegistry} * objects. * @param ioExecutor Executor used to run (async) deletes. @@ -43,7 +42,6 @@ public interface CheckpointRecoveryFactory { CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, - ClassLoader userClassLoader, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor) throws Exception; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreUtils.java similarity index 74% rename from flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreUtils.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreUtils.java index 0c9d54782f2a9..b9ba3c9bb3729 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreUtils.java @@ -19,6 +19,8 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.persistence.ResourceVersion; import org.apache.flink.runtime.persistence.StateHandleStore; import org.apache.flink.runtime.state.RetrievableStateHandle; @@ -37,15 +39,43 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** Helper methods related to {@link DefaultCompletedCheckpointStore}. */ -public class DefaultCompletedCheckpointStoreUtils { +public class CompletedCheckpointStoreUtils { - private static final Logger LOG = - LoggerFactory.getLogger(DefaultCompletedCheckpointStoreUtils.class); + private static final Logger LOG = LoggerFactory.getLogger(CompletedCheckpointStoreUtils.class); - private DefaultCompletedCheckpointStoreUtils() { + private CompletedCheckpointStoreUtils() { // No-op. } + /** + * Extracts maximum number of retained checkpoints configuration from the passed {@link + * Configuration}. The default value is used as a fallback if the passed value is a value larger + * than {@code 0}. + * + * @param config The configuration that is accessed. + * @param logger The {@link Logger} used for exposing the warning if the configured value is + * invalid. + * @return The maximum number of retained checkpoints based on the passed {@code Configuration}. + */ + public static int getMaximumNumberOfRetainedCheckpoints(Configuration config, Logger logger) { + final int maxNumberOfCheckpointsToRetain = + config.getInteger(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS); + + if (maxNumberOfCheckpointsToRetain <= 0) { + // warning and use 1 as the default value if the setting in + // state.checkpoints.max-retained-checkpoints is not greater than 0. + logger.warn( + "The setting for '{} : {}' is invalid. Using default value of {}", + CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.key(), + maxNumberOfCheckpointsToRetain, + CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.defaultValue()); + + return CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.defaultValue(); + } + + return maxNumberOfCheckpointsToRetain; + } + /** * Fetch all {@link CompletedCheckpoint completed checkpoints} from an {@link StateHandleStore * external store}. This method is intended for retrieving an initial state of {@link diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java index 83ce07732e108..bc18c453969d1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java @@ -74,7 +74,6 @@ public PerJobCheckpointRecoveryFactory( public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, - ClassLoader userClassLoader, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor) { return store.compute( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java index a77c6fb8cb372..95f9da72406ff 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java @@ -31,7 +31,6 @@ public class StandaloneCheckpointRecoveryFactory implements CheckpointRecoveryFa public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, - ClassLoader userClassLoader, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor) throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java index db4cfeceeb24f..354259bd81953 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java @@ -50,7 +50,6 @@ public ZooKeeperCheckpointRecoveryFactory( public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, - ClassLoader userClassLoader, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor) throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java index a971e69a456d6..2636ee6b7906e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java @@ -20,32 +20,45 @@ import org.apache.flink.api.common.JobStatus; +import org.apache.flink.shaded.guava30.com.google.common.collect.BiMap; +import org.apache.flink.shaded.guava30.com.google.common.collect.EnumBiMap; + /** The status of an application. */ public enum ApplicationStatus { - /** Application finished successfully */ + /** Application finished successfully. */ SUCCEEDED(0), - /** Application encountered an unrecoverable failure or error */ + /** Application encountered an unrecoverable failure or error. */ FAILED(1443), - /** Application was canceled or killed on request */ + /** Application was canceled or killed on request. */ CANCELED(0), - /** Application status is not known */ + /** Application status is not known. */ UNKNOWN(1445); // ------------------------------------------------------------------------ - /** The associated process exit code */ + private static final BiMap JOB_STATUS_APPLICATION_STATUS_BI_MAP = + EnumBiMap.create(JobStatus.class, ApplicationStatus.class); + + static { + // only globally-terminated JobStatus have a corresponding ApplicationStatus + JOB_STATUS_APPLICATION_STATUS_BI_MAP.put(JobStatus.FAILED, ApplicationStatus.FAILED); + JOB_STATUS_APPLICATION_STATUS_BI_MAP.put(JobStatus.CANCELED, ApplicationStatus.CANCELED); + JOB_STATUS_APPLICATION_STATUS_BI_MAP.put(JobStatus.FINISHED, ApplicationStatus.SUCCEEDED); + } + + /** The associated process exit code. */ private final int processExitCode; - private ApplicationStatus(int exitCode) { + ApplicationStatus(int exitCode) { this.processExitCode = exitCode; } /** - * Gets the process exit code associated with this status + * Gets the process exit code associated with this status. * * @return The associated process exit code. */ @@ -59,20 +72,21 @@ public int processExitCode() { * #UNKNOWN}. */ public static ApplicationStatus fromJobStatus(JobStatus jobStatus) { - if (jobStatus == null) { - return UNKNOWN; - } else { - switch (jobStatus) { - case FAILED: - return FAILED; - case CANCELED: - return CANCELED; - case FINISHED: - return SUCCEEDED; - - default: - return UNKNOWN; - } + return JOB_STATUS_APPLICATION_STATUS_BI_MAP.getOrDefault(jobStatus, UNKNOWN); + } + + /** + * Derives the {@link JobStatus} from the {@code ApplicationStatus}. + * + * @return The corresponding {@code JobStatus}. + * @throws UnsupportedOperationException for {@link #UNKNOWN}. + */ + public JobStatus deriveJobStatus() { + if (!JOB_STATUS_APPLICATION_STATUS_BI_MAP.inverse().containsKey(this)) { + throw new UnsupportedOperationException( + this.name() + " cannot be mapped to a JobStatus."); } + + return JOB_STATUS_APPLICATION_STATUS_BI_MAP.inverse().get(this); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/SubpartitionIndexRange.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/SubpartitionIndexRange.java index 19484a6cb3b67..5682f04ca92d8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/SubpartitionIndexRange.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/SubpartitionIndexRange.java @@ -27,7 +27,7 @@ public class SubpartitionIndexRange implements Serializable { private final int startIndex; private final int endIndex; - SubpartitionIndexRange(int startIndex, int endIndex) { + public SubpartitionIndexRange(int startIndex, int endIndex) { checkArgument(startIndex >= 0); checkArgument(endIndex >= startIndex); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index fdfc505351614..4e04c29d66ed1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.dispatcher; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.operators.ResourceSpec; @@ -32,11 +33,16 @@ import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.dispatcher.cleanup.CleanupRunnerFactory; +import org.apache.flink.runtime.dispatcher.cleanup.DispatcherResourceCleanerFactory; +import org.apache.flink.runtime.dispatcher.cleanup.ResourceCleaner; +import org.apache.flink.runtime.dispatcher.cleanup.ResourceCleanerFactory; import org.apache.flink.runtime.entrypoint.ClusterEntryPointExceptionUtils; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -89,6 +95,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; @@ -110,7 +117,7 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint resourceManagerGatewayRetriever; @@ -120,15 +127,18 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint runningJobs; + private final JobManagerRunnerRegistry jobManagerRunnerRegistry; private final Collection recoveredJobs; + private final Collection recoveredDirtyJobs; + private final DispatcherBootstrapFactory dispatcherBootstrapFactory; private final ExecutionGraphInfoStore executionGraphInfoStore; private final JobManagerRunnerFactory jobManagerRunnerFactory; + private final CleanupRunnerFactory cleanupRunnerFactory; private final JobManagerMetricGroup jobManagerMetricGroup; @@ -146,6 +156,9 @@ public abstract class Dispatcher extends PermanentlyFencedRpcEndpoint recoveredJobs, + Collection recoveredDirtyJobs, DispatcherBootstrapFactory dispatcherBootstrapFactory, DispatcherServices dispatcherServices) throws Exception { + this( + rpcService, + fencingToken, + recoveredJobs, + recoveredDirtyJobs, + dispatcherBootstrapFactory, + dispatcherServices, + new JobManagerRunnerRegistry(16)); + } + + private Dispatcher( + RpcService rpcService, + DispatcherId fencingToken, + Collection recoveredJobs, + Collection globallyTerminatedJobs, + DispatcherBootstrapFactory dispatcherBootstrapFactory, + DispatcherServices dispatcherServices, + JobManagerRunnerRegistry jobManagerRunnerRegistry) + throws Exception { + this( + rpcService, + fencingToken, + recoveredJobs, + globallyTerminatedJobs, + dispatcherServices.getConfiguration(), + dispatcherServices.getHighAvailabilityServices(), + dispatcherServices.getResourceManagerGatewayRetriever(), + dispatcherServices.getHeartbeatServices(), + dispatcherServices.getBlobServer(), + dispatcherServices.getFatalErrorHandler(), + dispatcherServices.getJobGraphWriter(), + dispatcherServices.getJobResultStore(), + dispatcherServices.getJobManagerMetricGroup(), + dispatcherServices.getMetricQueryServiceAddress(), + dispatcherServices.getIoExecutor(), + dispatcherServices.getHistoryServerArchivist(), + dispatcherServices.getArchivedExecutionGraphStore(), + dispatcherServices.getJobManagerRunnerFactory(), + dispatcherServices.getCleanupRunnerFactory(), + dispatcherBootstrapFactory, + dispatcherServices.getOperationCaches(), + jobManagerRunnerRegistry, + new DispatcherResourceCleanerFactory(jobManagerRunnerRegistry, dispatcherServices)); + } + + @VisibleForTesting + protected Dispatcher( + RpcService rpcService, + DispatcherId fencingToken, + Collection recoveredJobs, + Collection recoveredDirtyJobs, + Configuration configuration, + HighAvailabilityServices highAvailabilityServices, + GatewayRetriever resourceManagerGatewayRetriever, + HeartbeatServices heartbeatServices, + BlobServer blobServer, + FatalErrorHandler fatalErrorHandler, + JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore, + JobManagerMetricGroup jobManagerMetricGroup, + @Nullable String metricServiceQueryAddress, + Executor ioExecutor, + HistoryServerArchivist historyServerArchivist, + ExecutionGraphInfoStore executionGraphInfoStore, + JobManagerRunnerFactory jobManagerRunnerFactory, + CleanupRunnerFactory cleanupRunnerFactory, + DispatcherBootstrapFactory dispatcherBootstrapFactory, + DispatcherOperationCaches dispatcherOperationCaches, + JobManagerRunnerRegistry jobManagerRunnerRegistry, + ResourceCleanerFactory resourceCleanerFactory) + throws Exception { super(rpcService, RpcServiceUtils.createRandomName(DISPATCHER_NAME), fencingToken); - checkNotNull(dispatcherServices); - - this.configuration = dispatcherServices.getConfiguration(); - this.highAvailabilityServices = dispatcherServices.getHighAvailabilityServices(); - this.resourceManagerGatewayRetriever = - dispatcherServices.getResourceManagerGatewayRetriever(); - this.heartbeatServices = dispatcherServices.getHeartbeatServices(); - this.blobServer = dispatcherServices.getBlobServer(); - this.fatalErrorHandler = dispatcherServices.getFatalErrorHandler(); - this.jobGraphWriter = dispatcherServices.getJobGraphWriter(); - this.jobManagerMetricGroup = dispatcherServices.getJobManagerMetricGroup(); - this.metricServiceQueryAddress = dispatcherServices.getMetricQueryServiceAddress(); - this.ioExecutor = dispatcherServices.getIoExecutor(); + assertRecoveredJobsAndDirtyJobResults(recoveredJobs, recoveredDirtyJobs); + + this.configuration = checkNotNull(configuration); + this.highAvailabilityServices = checkNotNull(highAvailabilityServices); + this.resourceManagerGatewayRetriever = checkNotNull(resourceManagerGatewayRetriever); + this.heartbeatServices = checkNotNull(heartbeatServices); + this.blobServer = checkNotNull(blobServer); + this.fatalErrorHandler = checkNotNull(fatalErrorHandler); + this.jobGraphWriter = checkNotNull(jobGraphWriter); + this.jobResultStore = checkNotNull(jobResultStore); + this.jobManagerMetricGroup = checkNotNull(jobManagerMetricGroup); + this.metricServiceQueryAddress = metricServiceQueryAddress; + this.ioExecutor = checkNotNull(ioExecutor); this.jobManagerSharedServices = JobManagerSharedServices.fromConfiguration( configuration, blobServer, fatalErrorHandler); - this.runningJobsRegistry = highAvailabilityServices.getRunningJobsRegistry(); + this.jobManagerRunnerRegistry = checkNotNull(jobManagerRunnerRegistry); - runningJobs = new HashMap<>(16); + this.historyServerArchivist = checkNotNull(historyServerArchivist); - this.historyServerArchivist = dispatcherServices.getHistoryServerArchivist(); + this.executionGraphInfoStore = checkNotNull(executionGraphInfoStore); - this.executionGraphInfoStore = dispatcherServices.getArchivedExecutionGraphStore(); - - this.jobManagerRunnerFactory = dispatcherServices.getJobManagerRunnerFactory(); + this.jobManagerRunnerFactory = checkNotNull(jobManagerRunnerFactory); + this.cleanupRunnerFactory = checkNotNull(cleanupRunnerFactory); this.jobManagerRunnerTerminationFutures = new HashMap<>(2); @@ -195,14 +279,20 @@ public Dispatcher( this.dispatcherBootstrapFactory = checkNotNull(dispatcherBootstrapFactory); this.recoveredJobs = new HashSet<>(recoveredJobs); + + this.recoveredDirtyJobs = new HashSet<>(recoveredDirtyJobs); + this.blobServer.retainJobs( recoveredJobs.stream().map(JobGraph::getJobID).collect(Collectors.toSet())); this.dispatcherCachedOperationsHandler = new DispatcherCachedOperationsHandler( - dispatcherServices.getOperationCaches(), + checkNotNull(dispatcherOperationCaches), this::triggerSavepointAndGetLocation, this::stopWithSavepointAndGetLocation); + + this.localResourceCleaner = resourceCleanerFactory.createLocalResourceCleaner(); + this.globalResourceCleaner = resourceCleanerFactory.createGlobalResourceCleaner(); } // ------------------------------------------------------ @@ -229,7 +319,9 @@ public void onStart() throws Exception { throw exception; } + startCleanupRetries(); startRecoveredJobs(); + this.dispatcherBootstrap = this.dispatcherBootstrapFactory.create( getSelfGateway(DispatcherGateway.class), @@ -245,6 +337,25 @@ private void startDispatcherServices() throws Exception { } } + private static void assertRecoveredJobsAndDirtyJobResults( + Collection recoveredJobs, Collection recoveredDirtyJobResults) { + final Set jobIdsOfFinishedJobs = + recoveredDirtyJobResults.stream() + .map(JobResult::getJobId) + .collect(Collectors.toSet()); + + final boolean noRecoveredJobGraphHasDirtyJobResult = + recoveredJobs.stream() + .noneMatch( + recoveredJobGraph -> + jobIdsOfFinishedJobs.contains( + recoveredJobGraph.getJobID())); + + Preconditions.checkArgument( + noRecoveredJobGraphHasDirtyJobResult, + "There should be no overlap between the recovered JobGraphs and the passed dirty JobResults based on their job ID."); + } + private void startRecoveredJobs() { for (JobGraph recoveredJob : recoveredJobs) { runRecoveredJob(recoveredJob); @@ -255,7 +366,7 @@ private void startRecoveredJobs() { private void runRecoveredJob(final JobGraph recoveredJob) { checkNotNull(recoveredJob); try { - runJob(recoveredJob, ExecutionType.RECOVERY); + initializeAndStartJobManagerRunner(recoveredJob, ExecutionType.RECOVERY); } catch (Throwable throwable) { onFatalError( new DispatcherException( @@ -265,6 +376,26 @@ private void runRecoveredJob(final JobGraph recoveredJob) { } } + private void startCleanupRetries() { + recoveredDirtyJobs.forEach(this::runCleanupRetry); + recoveredDirtyJobs.clear(); + } + + private void runCleanupRetry(final JobResult jobResult) { + checkNotNull(jobResult); + + try { + initializeAndStartCheckpointJobDataCleanupRunner(jobResult); + } catch (Throwable throwable) { + onFatalError( + new DispatcherException( + String.format( + "Could not start cleanup retry for job %s.", + jobResult.getJobId()), + throwable)); + } + } + private void handleStartDispatcherServicesException(Exception e) throws Exception { try { stopDispatcherServices(); @@ -341,7 +472,7 @@ public CompletableFuture submitJob(JobGraph jobGraph, Time timeout) public CompletableFuture submitFailedJob( JobID jobId, String jobName, Throwable exception) { final ArchivedExecutionGraph archivedExecutionGraph = - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, jobName, JobStatus.FAILED, @@ -360,7 +491,7 @@ public CompletableFuture submitFailedJob( * @throws FlinkException if the job scheduling status cannot be retrieved */ private boolean isDuplicateJob(JobID jobId) throws FlinkException { - return isInGloballyTerminalState(jobId) || runningJobs.containsKey(jobId); + return isInGloballyTerminalState(jobId) || jobManagerRunnerRegistry.isRegistered(jobId); } /** @@ -372,9 +503,7 @@ private boolean isDuplicateJob(JobID jobId) throws FlinkException { */ private boolean isInGloballyTerminalState(JobID jobId) throws FlinkException { try { - final RunningJobsRegistry.JobSchedulingStatus schedulingStatus = - runningJobsRegistry.getJobSchedulingStatus(jobId); - return schedulingStatus == RunningJobsRegistry.JobSchedulingStatus.DONE; + return jobResultStore.hasJobResultEntry(jobId); } catch (IOException e) { throw new FlinkException( String.format("Failed to retrieve job scheduling status for job %s.", jobId), @@ -411,7 +540,16 @@ private CompletableFuture internalSubmitJob(JobGraph jobGraph) { return persistAndRunFuture.handleAsync( (acknowledge, throwable) -> { if (throwable != null) { - cleanUpHighAvailabilityJobData(jobGraph.getJobID()); + globalResourceCleaner + .cleanupAsync(jobGraph.getJobID()) + .exceptionally( + t -> { + log.warn( + "Cleanup didn't succeed after job submission failed for job " + + jobGraph.getJobID(), + t); + return null; + }); ClusterEntryPointExceptionUtils.tryEnrichClusterEntryPointError(throwable); final Throwable strippedThrowable = ExceptionUtils.stripCompletionException(throwable); @@ -431,18 +569,30 @@ private CompletableFuture internalSubmitJob(JobGraph jobGraph) { private void persistAndRunJob(JobGraph jobGraph) throws Exception { jobGraphWriter.putJobGraph(jobGraph); - runJob(jobGraph, ExecutionType.SUBMISSION); + initializeAndStartJobManagerRunner(jobGraph, ExecutionType.SUBMISSION); } - private void runJob(JobGraph jobGraph, ExecutionType executionType) throws Exception { - Preconditions.checkState(!runningJobs.containsKey(jobGraph.getJobID())); - long initializationTimestamp = System.currentTimeMillis(); - JobManagerRunner jobManagerRunner = - createJobManagerRunner(jobGraph, initializationTimestamp); + private void initializeAndStartJobManagerRunner(JobGraph jobGraph, ExecutionType executionType) + throws Exception { + Preconditions.checkState(!jobManagerRunnerRegistry.isRegistered(jobGraph.getJobID())); + final JobManagerRunner jobManagerRunner = initializeJobManagerRunner(jobGraph); + runJob(jobManagerRunner, executionType); + } - runningJobs.put(jobGraph.getJobID(), jobManagerRunner); + private void initializeAndStartCheckpointJobDataCleanupRunner(JobResult jobResult) + throws Exception { + Preconditions.checkState(!jobManagerRunnerRegistry.isRegistered(jobResult.getJobId())); + final JobManagerRunner checkpointJobDataCleanupRunner = + initializeCheckpointJobDataCleanupRunner(jobResult); + runJob(checkpointJobDataCleanupRunner, ExecutionType.RECOVERY); + } + + private void runJob(JobManagerRunner jobManagerRunner, ExecutionType executionType) + throws Exception { + jobManagerRunner.start(); + jobManagerRunnerRegistry.register(jobManagerRunner); - final JobID jobId = jobGraph.getJobID(); + final JobID jobId = jobManagerRunner.getJobID(); final CompletableFuture cleanupJobStateFuture = jobManagerRunner @@ -450,7 +600,9 @@ private void runJob(JobGraph jobGraph, ExecutionType executionType) throws Excep .handleAsync( (jobManagerRunnerResult, throwable) -> { Preconditions.checkState( - runningJobs.get(jobId) == jobManagerRunner, + jobManagerRunnerRegistry.isRegistered(jobId) + && jobManagerRunnerRegistry.get(jobId) + == jobManagerRunner, "The job entry in runningJobs must be bound to the lifetime of the JobManagerRunner."); if (jobManagerRunnerResult != null) { @@ -475,28 +627,18 @@ private void runJob(JobGraph jobGraph, ExecutionType executionType) throws Excep private CleanupJobState handleJobManagerRunnerResult( JobManagerRunnerResult jobManagerRunnerResult, ExecutionType executionType) { - if (jobManagerRunnerResult.isInitializationFailure()) { - if (executionType == ExecutionType.RECOVERY) { - return jobManagerRunnerFailed( - jobManagerRunnerResult.getExecutionGraphInfo().getJobId(), - jobManagerRunnerResult.getInitializationFailure()); - } else { - return jobReachedTerminalState(jobManagerRunnerResult.getExecutionGraphInfo()); - } - } else { - return jobReachedTerminalState(jobManagerRunnerResult.getExecutionGraphInfo()); + if (jobManagerRunnerResult.isInitializationFailure() + && executionType == ExecutionType.RECOVERY) { + return jobManagerRunnerFailed( + jobManagerRunnerResult.getExecutionGraphInfo().getJobId(), + jobManagerRunnerResult.getInitializationFailure()); } + return jobReachedTerminalState(jobManagerRunnerResult.getExecutionGraphInfo()); } enum CleanupJobState { - LOCAL(false), - GLOBAL(true); - - final boolean cleanupHAData; - - CleanupJobState(boolean cleanupHAData) { - this.cleanupHAData = cleanupHAData; - } + LOCAL, + GLOBAL } private CleanupJobState jobManagerRunnerFailed(JobID jobId, Throwable throwable) { @@ -504,29 +646,34 @@ private CleanupJobState jobManagerRunnerFailed(JobID jobId, Throwable throwable) return CleanupJobState.LOCAL; } - JobManagerRunner createJobManagerRunner(JobGraph jobGraph, long initializationTimestamp) - throws Exception { + private JobManagerRunner initializeJobManagerRunner(JobGraph jobGraph) throws Exception { final RpcService rpcService = getRpcService(); - JobManagerRunner runner = - jobManagerRunnerFactory.createJobManagerRunner( - jobGraph, - configuration, - rpcService, - highAvailabilityServices, - heartbeatServices, - jobManagerSharedServices, - new DefaultJobManagerJobMetricGroupFactory(jobManagerMetricGroup), - fatalErrorHandler, - initializationTimestamp); - runner.start(); - return runner; + return jobManagerRunnerFactory.createJobManagerRunner( + jobGraph, + configuration, + rpcService, + highAvailabilityServices, + heartbeatServices, + jobManagerSharedServices, + new DefaultJobManagerJobMetricGroupFactory(jobManagerMetricGroup), + fatalErrorHandler, + System.currentTimeMillis()); + } + + private JobManagerRunner initializeCheckpointJobDataCleanupRunner(JobResult jobResult) + throws Exception { + return cleanupRunnerFactory.create( + jobResult, + highAvailabilityServices.getCheckpointRecoveryFactory(), + configuration, + ioExecutor); } @Override public CompletableFuture> listJobs(Time timeout) { return CompletableFuture.completedFuture( - Collections.unmodifiableSet(new HashSet<>(runningJobs.keySet()))); + Collections.unmodifiableSet(jobManagerRunnerRegistry.getRunningJobIds())); } @Override @@ -671,9 +818,7 @@ public CompletableFuture requestExecutionGraphInfo( @Override public CompletableFuture requestJobResult(JobID jobId, Time timeout) { - JobManagerRunner job = runningJobs.get(jobId); - - if (job == null) { + if (!jobManagerRunnerRegistry.isRegistered(jobId)) { final ExecutionGraphInfo executionGraphInfo = executionGraphInfoStore.get(jobId); if (executionGraphInfo == null) { @@ -682,15 +827,17 @@ public CompletableFuture requestJobResult(JobID jobId, Time timeout) return CompletableFuture.completedFuture( JobResult.createFrom(executionGraphInfo.getArchivedExecutionGraph())); } - } else { - return job.getResultFuture() - .thenApply( - jobManagerRunnerResult -> - JobResult.createFrom( - jobManagerRunnerResult - .getExecutionGraphInfo() - .getArchivedExecutionGraph())); } + + final JobManagerRunner jobManagerRunner = jobManagerRunnerRegistry.get(jobId); + return jobManagerRunner + .getResultFuture() + .thenApply( + jobManagerRunnerResult -> + JobResult.createFrom( + jobManagerRunnerResult + .getExecutionGraphInfo() + .getArchivedExecutionGraph())); } @Override @@ -824,77 +971,33 @@ private void registerJobManagerRunnerTerminationFuture( } private CompletableFuture removeJob(JobID jobId, CleanupJobState cleanupJobState) { - final JobManagerRunner job = checkNotNull(runningJobs.remove(jobId)); - return CompletableFuture.supplyAsync( - () -> cleanUpJobGraph(jobId, cleanupJobState.cleanupHAData), ioExecutor) - .thenCompose( - jobGraphRemoved -> job.closeAsync().thenApply(ignored -> jobGraphRemoved)) - .thenAcceptAsync( - jobGraphRemoved -> cleanUpRemainingJobData(jobId, jobGraphRemoved), - ioExecutor); - } - - /** - * Clean up job graph from {@link org.apache.flink.runtime.jobmanager.JobGraphStore}. - * - * @param jobId Reference to the job that we want to clean. - * @param cleanupHA Flag signalling whether we should remove (we're done with the job) or just - * release the job graph. - * @return True if we have removed the job graph. This means we can clean other HA-related - * services as well. - */ - private boolean cleanUpJobGraph(JobID jobId, boolean cleanupHA) { - if (cleanupHA) { - try { - jobGraphWriter.removeJobGraph(jobId); - return true; - } catch (Exception e) { - log.warn( - "Could not properly remove job {} from submitted job graph store.", - jobId, - e); - return false; - } - } - try { - jobGraphWriter.releaseJobGraph(jobId); - } catch (Exception e) { - log.warn("Could not properly release job {} from submitted job graph store.", jobId, e); + switch (cleanupJobState) { + case LOCAL: + return localResourceCleaner.cleanupAsync(jobId); + case GLOBAL: + return globalResourceCleaner + .cleanupAsync(jobId) + .thenRun(() -> markJobAsClean(jobId)); + default: + throw new IllegalStateException("Invalid cleanup state: " + cleanupJobState); } - return false; } - private void cleanUpRemainingJobData(JobID jobId, boolean jobGraphRemoved) { - jobManagerMetricGroup.removeJob(jobId); - if (jobGraphRemoved) { - try { - runningJobsRegistry.clearJob(jobId); - } catch (IOException e) { - log.warn( - "Could not properly remove job {} from the running jobs registry.", - jobId, - e); - } - try { - highAvailabilityServices.cleanupJobData(jobId); - } catch (Exception e) { - log.warn( - "Could not properly clean data for job {} stored by ha services", jobId, e); - } + private void markJobAsClean(JobID jobId) { + try { + jobResultStore.markResultAsClean(jobId); + log.debug( + "Cleanup for the job '{}' has finished. Job has been marked as clean.", jobId); + } catch (IOException e) { + log.warn("Could not properly mark job {} result as clean.", jobId, e); } - blobServer.cleanupJob(jobId, jobGraphRemoved); - } - - private void cleanUpHighAvailabilityJobData(JobID jobId) { - final boolean jobGraphRemoved = cleanUpJobGraph(jobId, true); - cleanUpRemainingJobData(jobId, jobGraphRemoved); } /** Terminate all currently running {@link JobManagerRunner}s. */ private void terminateRunningJobs() { log.info("Stopping all currently running jobs of dispatcher {}.", getAddress()); - final HashSet jobsToRemove = new HashSet<>(runningJobs.keySet()); + final Set jobsToRemove = jobManagerRunnerRegistry.getRunningJobIds(); for (JobID jobId : jobsToRemove) { terminateJob(jobId); @@ -902,9 +1005,8 @@ private void terminateRunningJobs() { } private void terminateJob(JobID jobId) { - final JobManagerRunner jobManagerRunner = runningJobs.get(jobId); - - if (jobManagerRunner != null) { + if (jobManagerRunnerRegistry.isRegistered(jobId)) { + final JobManagerRunner jobManagerRunner = jobManagerRunnerRegistry.get(jobId); jobManagerRunner.closeAsync(); } } @@ -952,6 +1054,29 @@ protected CleanupJobState jobReachedTerminalState(ExecutionGraphInfo executionGr archiveExecutionGraph(executionGraphInfo); + if (terminalJobStatus.isGloballyTerminalState()) { + final JobID jobId = executionGraphInfo.getJobId(); + try { + if (jobResultStore.hasCleanJobResultEntry(jobId)) { + log.warn( + "Job {} is already marked as clean but clean up was triggered again.", + jobId); + } else if (!jobResultStore.hasDirtyJobResultEntry(jobId)) { + jobResultStore.createDirtyResult( + new JobResultEntry( + JobResult.createFrom( + executionGraphInfo.getArchivedExecutionGraph()))); + } + } catch (IOException e) { + fatalErrorHandler.onFatalError( + new FlinkException( + String.format( + "The job %s couldn't be marked as pre-cleanup finished in JobResultStore.", + jobId), + e)); + } + } + return terminalJobStatus.isGloballyTerminalState() ? CleanupJobState.GLOBAL : CleanupJobState.LOCAL; @@ -996,11 +1121,11 @@ private void jobMasterFailed(JobID jobId, Throwable cause) { /** Ensures that the JobMasterGateway is available. */ private CompletableFuture getJobMasterGateway(JobID jobId) { - JobManagerRunner job = runningJobs.get(jobId); - if (job == null) { + if (!jobManagerRunnerRegistry.isRegistered(jobId)) { return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId)); } + final JobManagerRunner job = jobManagerRunnerRegistry.get(jobId); if (!job.isInitialized()) { return FutureUtils.completedExceptionally( new UnavailableDispatcherOperationException( @@ -1020,7 +1145,9 @@ private CompletableFuture getResourceManagerGateway() { } private Optional getJobManagerRunner(JobID jobId) { - return Optional.ofNullable(runningJobs.get(jobId)); + return jobManagerRunnerRegistry.isRegistered(jobId) + ? Optional.of(jobManagerRunnerRegistry.get(jobId)) + : Optional.empty(); } private CompletableFuture runResourceManagerCommand( @@ -1042,9 +1169,9 @@ private List>> queryJobMastersForInformation( Function> queryFunction) { List>> optionalJobInformation = - new ArrayList<>(runningJobs.size()); + new ArrayList<>(jobManagerRunnerRegistry.size()); - for (JobManagerRunner job : runningJobs.values()) { + for (JobManagerRunner job : jobManagerRunnerRegistry.getJobManagerRunners()) { final CompletableFuture> queryResult = queryFunction .apply(job) @@ -1078,7 +1205,7 @@ private CompletableFuture waitForTerminatingJob( } CompletableFuture getJobTerminationFuture(JobID jobId) { - if (runningJobs.containsKey(jobId)) { + if (jobManagerRunnerRegistry.isRegistered(jobId)) { return FutureUtils.completedExceptionally( new DispatcherException( String.format("Job with job id %s is still running.", jobId))); @@ -1089,7 +1216,8 @@ CompletableFuture getJobTerminationFuture(JobID jobId) { } private void registerDispatcherMetrics(MetricGroup jobManagerMetricGroup) { - jobManagerMetricGroup.gauge(MetricNames.NUM_RUNNING_JOBS, () -> (long) runningJobs.size()); + jobManagerMetricGroup.gauge( + MetricNames.NUM_RUNNING_JOBS, () -> (long) jobManagerRunnerRegistry.size()); } public CompletableFuture onRemovedJobGraph(JobID jobId) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java index b9a7d7bb7c660..547bda32de2c3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.dispatcher; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; import java.util.Collection; @@ -31,7 +32,9 @@ Dispatcher createDispatcher( RpcService rpcService, DispatcherId fencingToken, Collection recoveredJobs, + Collection recoveredDirtyJobResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, - PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) + PartialDispatcherServicesWithJobPersistenceComponents + partialDispatcherServicesWithJobPersistenceComponents) throws Exception; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java index b18321c44e69a..12112b15b83e7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherServices.java @@ -20,15 +20,17 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.dispatcher.cleanup.CleanupRunnerFactory; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.util.Preconditions; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.concurrent.Executor; @@ -36,106 +38,113 @@ /** {@link Dispatcher} services container. */ public class DispatcherServices { - @Nonnull private final Configuration configuration; + private final Configuration configuration; - @Nonnull private final HighAvailabilityServices highAvailabilityServices; + private final HighAvailabilityServices highAvailabilityServices; - @Nonnull private final GatewayRetriever resourceManagerGatewayRetriever; + private final GatewayRetriever resourceManagerGatewayRetriever; - @Nonnull private final BlobServer blobServer; + private final BlobServer blobServer; - @Nonnull private final HeartbeatServices heartbeatServices; + private final HeartbeatServices heartbeatServices; - @Nonnull private final JobManagerMetricGroup jobManagerMetricGroup; + private final JobManagerMetricGroup jobManagerMetricGroup; - @Nonnull private final ExecutionGraphInfoStore executionGraphInfoStore; + private final ExecutionGraphInfoStore executionGraphInfoStore; - @Nonnull private final FatalErrorHandler fatalErrorHandler; + private final FatalErrorHandler fatalErrorHandler; - @Nonnull private final HistoryServerArchivist historyServerArchivist; + private final HistoryServerArchivist historyServerArchivist; @Nullable private final String metricQueryServiceAddress; - @Nonnull private final DispatcherOperationCaches operationCaches; + private final DispatcherOperationCaches operationCaches; - @Nonnull private final JobGraphWriter jobGraphWriter; + private final JobGraphWriter jobGraphWriter; - @Nonnull private final JobManagerRunnerFactory jobManagerRunnerFactory; + private final JobResultStore jobResultStore; - @Nonnull private final Executor ioExecutor; + private final JobManagerRunnerFactory jobManagerRunnerFactory; + + private final CleanupRunnerFactory cleanupRunnerFactory; + + private final Executor ioExecutor; DispatcherServices( - @Nonnull Configuration configuration, - @Nonnull HighAvailabilityServices highAvailabilityServices, - @Nonnull GatewayRetriever resourceManagerGatewayRetriever, - @Nonnull BlobServer blobServer, - @Nonnull HeartbeatServices heartbeatServices, - @Nonnull ExecutionGraphInfoStore executionGraphInfoStore, - @Nonnull FatalErrorHandler fatalErrorHandler, - @Nonnull HistoryServerArchivist historyServerArchivist, + Configuration configuration, + HighAvailabilityServices highAvailabilityServices, + GatewayRetriever resourceManagerGatewayRetriever, + BlobServer blobServer, + HeartbeatServices heartbeatServices, + ExecutionGraphInfoStore executionGraphInfoStore, + FatalErrorHandler fatalErrorHandler, + HistoryServerArchivist historyServerArchivist, @Nullable String metricQueryServiceAddress, - @Nonnull DispatcherOperationCaches operationCaches, - @Nonnull JobManagerMetricGroup jobManagerMetricGroup, - @Nonnull JobGraphWriter jobGraphWriter, - @Nonnull JobManagerRunnerFactory jobManagerRunnerFactory, - @Nonnull Executor ioExecutor) { - this.configuration = configuration; - this.highAvailabilityServices = highAvailabilityServices; - this.resourceManagerGatewayRetriever = resourceManagerGatewayRetriever; - this.blobServer = blobServer; - this.heartbeatServices = heartbeatServices; - this.executionGraphInfoStore = executionGraphInfoStore; - this.fatalErrorHandler = fatalErrorHandler; - this.historyServerArchivist = historyServerArchivist; + DispatcherOperationCaches operationCaches, + JobManagerMetricGroup jobManagerMetricGroup, + JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore, + JobManagerRunnerFactory jobManagerRunnerFactory, + CleanupRunnerFactory cleanupRunnerFactory, + Executor ioExecutor) { + this.configuration = Preconditions.checkNotNull(configuration, "Configuration"); + this.highAvailabilityServices = + Preconditions.checkNotNull(highAvailabilityServices, "HighAvailabilityServices"); + this.resourceManagerGatewayRetriever = + Preconditions.checkNotNull( + resourceManagerGatewayRetriever, "ResourceManagerGatewayRetriever"); + this.blobServer = Preconditions.checkNotNull(blobServer, "BlobServer"); + this.heartbeatServices = Preconditions.checkNotNull(heartbeatServices, "HeartBeatServices"); + this.executionGraphInfoStore = + Preconditions.checkNotNull(executionGraphInfoStore, "ExecutionGraphInfoStore"); + this.fatalErrorHandler = Preconditions.checkNotNull(fatalErrorHandler, "FatalErrorHandler"); + this.historyServerArchivist = + Preconditions.checkNotNull(historyServerArchivist, "HistoryServerArchivist"); this.metricQueryServiceAddress = metricQueryServiceAddress; - this.operationCaches = operationCaches; - this.jobManagerMetricGroup = jobManagerMetricGroup; - this.jobGraphWriter = jobGraphWriter; - this.jobManagerRunnerFactory = jobManagerRunnerFactory; - this.ioExecutor = ioExecutor; + this.operationCaches = Preconditions.checkNotNull(operationCaches, "OperationCaches"); + this.jobManagerMetricGroup = + Preconditions.checkNotNull(jobManagerMetricGroup, "JobManagerMetricGroup"); + this.jobGraphWriter = Preconditions.checkNotNull(jobGraphWriter, "JobGraphWriter"); + this.jobResultStore = Preconditions.checkNotNull(jobResultStore, "JobResultStore"); + this.jobManagerRunnerFactory = + Preconditions.checkNotNull(jobManagerRunnerFactory, "JobManagerRunnerFactory"); + this.cleanupRunnerFactory = + Preconditions.checkNotNull(cleanupRunnerFactory, "CleanupRunnerFactory"); + this.ioExecutor = Preconditions.checkNotNull(ioExecutor, "IOExecutor"); } - @Nonnull public Configuration getConfiguration() { return configuration; } - @Nonnull public HighAvailabilityServices getHighAvailabilityServices() { return highAvailabilityServices; } - @Nonnull public GatewayRetriever getResourceManagerGatewayRetriever() { return resourceManagerGatewayRetriever; } - @Nonnull public BlobServer getBlobServer() { return blobServer; } - @Nonnull public HeartbeatServices getHeartbeatServices() { return heartbeatServices; } - @Nonnull public JobManagerMetricGroup getJobManagerMetricGroup() { return jobManagerMetricGroup; } - @Nonnull public ExecutionGraphInfoStore getArchivedExecutionGraphStore() { return executionGraphInfoStore; } - @Nonnull public FatalErrorHandler getFatalErrorHandler() { return fatalErrorHandler; } - @Nonnull public HistoryServerArchivist getHistoryServerArchivist() { return historyServerArchivist; } @@ -145,47 +154,56 @@ public String getMetricQueryServiceAddress() { return metricQueryServiceAddress; } - @Nonnull public DispatcherOperationCaches getOperationCaches() { return operationCaches; } - @Nonnull public JobGraphWriter getJobGraphWriter() { return jobGraphWriter; } - @Nonnull + public JobResultStore getJobResultStore() { + return jobResultStore; + } + JobManagerRunnerFactory getJobManagerRunnerFactory() { return jobManagerRunnerFactory; } - @Nonnull + CleanupRunnerFactory getCleanupRunnerFactory() { + return cleanupRunnerFactory; + } + public Executor getIoExecutor() { return ioExecutor; } public static DispatcherServices from( - @Nonnull - PartialDispatcherServicesWithJobGraphStore - partialDispatcherServicesWithJobGraphStore, - @Nonnull JobManagerRunnerFactory jobManagerRunnerFactory) { + PartialDispatcherServicesWithJobPersistenceComponents + partialDispatcherServicesWithJobPersistenceComponents, + JobManagerRunnerFactory jobManagerRunnerFactory, + CleanupRunnerFactory cleanupRunnerFactory) { return new DispatcherServices( - partialDispatcherServicesWithJobGraphStore.getConfiguration(), - partialDispatcherServicesWithJobGraphStore.getHighAvailabilityServices(), - partialDispatcherServicesWithJobGraphStore.getResourceManagerGatewayRetriever(), - partialDispatcherServicesWithJobGraphStore.getBlobServer(), - partialDispatcherServicesWithJobGraphStore.getHeartbeatServices(), - partialDispatcherServicesWithJobGraphStore.getArchivedExecutionGraphStore(), - partialDispatcherServicesWithJobGraphStore.getFatalErrorHandler(), - partialDispatcherServicesWithJobGraphStore.getHistoryServerArchivist(), - partialDispatcherServicesWithJobGraphStore.getMetricQueryServiceAddress(), - partialDispatcherServicesWithJobGraphStore.getOperationCaches(), - partialDispatcherServicesWithJobGraphStore + partialDispatcherServicesWithJobPersistenceComponents.getConfiguration(), + partialDispatcherServicesWithJobPersistenceComponents.getHighAvailabilityServices(), + partialDispatcherServicesWithJobPersistenceComponents + .getResourceManagerGatewayRetriever(), + partialDispatcherServicesWithJobPersistenceComponents.getBlobServer(), + partialDispatcherServicesWithJobPersistenceComponents.getHeartbeatServices(), + partialDispatcherServicesWithJobPersistenceComponents + .getArchivedExecutionGraphStore(), + partialDispatcherServicesWithJobPersistenceComponents.getFatalErrorHandler(), + partialDispatcherServicesWithJobPersistenceComponents.getHistoryServerArchivist(), + partialDispatcherServicesWithJobPersistenceComponents + .getMetricQueryServiceAddress(), + partialDispatcherServicesWithJobPersistenceComponents.getOperationCaches(), + partialDispatcherServicesWithJobPersistenceComponents .getJobManagerMetricGroupFactory() .create(), - partialDispatcherServicesWithJobGraphStore.getJobGraphWriter(), + partialDispatcherServicesWithJobPersistenceComponents.getJobGraphWriter(), + partialDispatcherServicesWithJobPersistenceComponents.getJobResultStore(), jobManagerRunnerFactory, - partialDispatcherServicesWithJobGraphStore.getIoExecutor()); + cleanupRunnerFactory, + partialDispatcherServicesWithJobPersistenceComponents.getIoExecutor()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java index 82c225b3a554b..ba800e9ff2a16 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobDispatcherFactory.java @@ -19,9 +19,12 @@ package org.apache.flink.runtime.dispatcher; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.dispatcher.cleanup.CheckpointResourcesCleanupRunnerFactory; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; @@ -38,13 +41,21 @@ public MiniDispatcher createDispatcher( RpcService rpcService, DispatcherId fencingToken, Collection recoveredJobs, + Collection recoveredDirtyJobResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, - PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) + PartialDispatcherServicesWithJobPersistenceComponents + partialDispatcherServicesWithJobPersistenceComponents) throws Exception { - final JobGraph jobGraph = Iterables.getOnlyElement(recoveredJobs); + final JobGraph recoveredJobGraph = Iterables.getOnlyElement(recoveredJobs, null); + final JobResult recoveredDirtyJob = + Iterables.getOnlyElement(recoveredDirtyJobResults, null); + + Preconditions.checkArgument( + recoveredJobGraph == null ^ recoveredDirtyJob == null, + "Either the JobGraph or the recovered JobResult needs to be specified."); final Configuration configuration = - partialDispatcherServicesWithJobGraphStore.getConfiguration(); + partialDispatcherServicesWithJobPersistenceComponents.getConfiguration(); final String executionModeValue = configuration.getString(INTERNAL_CLUSTER_EXECUTION_MODE); final ClusterEntrypoint.ExecutionMode executionMode = ClusterEntrypoint.ExecutionMode.valueOf(executionModeValue); @@ -53,9 +64,11 @@ public MiniDispatcher createDispatcher( rpcService, fencingToken, DispatcherServices.from( - partialDispatcherServicesWithJobGraphStore, - JobMasterServiceLeadershipRunnerFactory.INSTANCE), - jobGraph, + partialDispatcherServicesWithJobPersistenceComponents, + JobMasterServiceLeadershipRunnerFactory.INSTANCE, + CheckpointResourcesCleanupRunnerFactory.INSTANCE), + recoveredJobGraph, + recoveredDirtyJob, dispatcherBootstrapFactory, executionMode); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobManagerRunnerRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobManagerRunnerRegistry.java new file mode 100644 index 0000000000000..dc9f2f4b6e4e8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobManagerRunnerRegistry.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.dispatcher.cleanup.GloballyCleanableResource; +import org.apache.flink.runtime.dispatcher.cleanup.LocallyCleanableResource; +import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.concurrent.FutureUtils; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; + +/** {@code JobManagerRunner} collects running jobs represented by {@link JobManagerRunner}. */ +public class JobManagerRunnerRegistry + implements LocallyCleanableResource, GloballyCleanableResource { + + @VisibleForTesting final Map jobManagerRunners; + + public JobManagerRunnerRegistry(int initialCapacity) { + Preconditions.checkArgument(initialCapacity > 0); + jobManagerRunners = new HashMap<>(initialCapacity); + } + + public boolean isRegistered(JobID jobId) { + return jobManagerRunners.containsKey(jobId); + } + + public void register(JobManagerRunner jobManagerRunner) { + Preconditions.checkArgument( + !isRegistered(jobManagerRunner.getJobID()), + "A job with the ID %s is already registered.", + jobManagerRunner.getJobID()); + this.jobManagerRunners.put(jobManagerRunner.getJobID(), jobManagerRunner); + } + + /** + * Returns the {@link JobManagerRunner} for the given {@code JobID}. + * + * @throws NoSuchElementException if the passed {@code JobID} does not belong to a registered + * {@code JobManagerRunner}. + * @see #isRegistered(JobID) + */ + public JobManagerRunner get(JobID jobId) { + assertJobRegistered(jobId); + return this.jobManagerRunners.get(jobId); + } + + public int size() { + return this.jobManagerRunners.size(); + } + + public Set getRunningJobIds() { + return this.jobManagerRunners.keySet(); + } + + public Collection getJobManagerRunners() { + return this.jobManagerRunners.values(); + } + + @Override + public void globalCleanup(JobID jobId) throws Exception { + cleanup(jobId); + } + + @Override + public CompletableFuture globalCleanupAsync(JobID jobId, Executor unusedExecutor) { + return cleanupAsync(jobId); + } + + @Override + public void localCleanup(JobID jobId) throws Exception { + cleanup(jobId); + } + + @Override + public CompletableFuture localCleanupAsync(JobID jobId, Executor unusedExecutor) { + return cleanupAsync(jobId); + } + + private void cleanup(JobID jobId) throws Exception { + if (isRegistered(jobId)) { + unregister(jobId).close(); + } + } + + private CompletableFuture cleanupAsync(JobID jobId) { + if (!isRegistered(jobId)) { + return FutureUtils.completedVoidFuture(); + } + + return unregister(jobId).closeAsync(); + } + + public JobManagerRunner unregister(JobID jobId) { + assertJobRegistered(jobId); + return this.jobManagerRunners.remove(jobId); + } + + private void assertJobRegistered(JobID jobId) { + if (!isRegistered(jobId)) { + throw new NoSuchElementException( + "There is no running job registered for the job ID " + jobId); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobMasterServiceLeadershipRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobMasterServiceLeadershipRunnerFactory.java index f025261030a18..e670f2d0cdd1e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobMasterServiceLeadershipRunnerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/JobMasterServiceLeadershipRunnerFactory.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.DefaultSlotPoolServiceSchedulerFactory; import org.apache.flink.runtime.jobmaster.JobManagerRunner; @@ -64,8 +64,8 @@ public JobManagerRunner createJobManagerRunner( final JobMasterConfiguration jobMasterConfiguration = JobMasterConfiguration.fromConfiguration(configuration); - final RunningJobsRegistry runningJobsRegistry = - highAvailabilityServices.getRunningJobsRegistry(); + final JobResultStore jobResultStore = highAvailabilityServices.getJobResultStore(); + final LeaderElectionService jobManagerLeaderElectionService = highAvailabilityServices.getJobManagerLeaderElectionService(jobGraph.getJobID()); @@ -118,7 +118,7 @@ public JobManagerRunner createJobManagerRunner( return new JobMasterServiceLeadershipRunner( jobMasterServiceProcessFactory, jobManagerLeaderElectionService, - runningJobsRegistry, + jobResultStore, classLoaderLease, fatalErrorHandler); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java index 83f173db8c750..bc5776e1948e6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java @@ -29,9 +29,11 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.FlinkException; -import java.util.Collections; +import javax.annotation.Nullable; + import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -55,14 +57,16 @@ public MiniDispatcher( RpcService rpcService, DispatcherId fencingToken, DispatcherServices dispatcherServices, - JobGraph jobGraph, + @Nullable JobGraph jobGraph, + @Nullable JobResult recoveredDirtyJob, DispatcherBootstrapFactory dispatcherBootstrapFactory, JobClusterEntrypoint.ExecutionMode executionMode) throws Exception { super( rpcService, fencingToken, - Collections.singleton(jobGraph), + CollectionUtil.ofNullable(jobGraph), + CollectionUtil.ofNullable(recoveredDirtyJob), dispatcherBootstrapFactory, dispatcherServices); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobPersistenceComponents.java similarity index 67% rename from flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobPersistenceComponents.java index c3fbecdc9c2b2..c6fe7b7eaa579 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobPersistenceComponents.java @@ -22,35 +22,38 @@ import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.concurrent.Executor; /** {@link DispatcherFactory} services container. */ -public class PartialDispatcherServicesWithJobGraphStore extends PartialDispatcherServices { +public class PartialDispatcherServicesWithJobPersistenceComponents + extends PartialDispatcherServices { - @Nonnull private final JobGraphWriter jobGraphWriter; + private final JobGraphWriter jobGraphWriter; + private final JobResultStore jobResultStore; - private PartialDispatcherServicesWithJobGraphStore( - @Nonnull Configuration configuration, - @Nonnull HighAvailabilityServices highAvailabilityServices, - @Nonnull GatewayRetriever resourceManagerGatewayRetriever, - @Nonnull BlobServer blobServer, - @Nonnull HeartbeatServices heartbeatServices, - @Nonnull JobManagerMetricGroupFactory jobManagerMetricGroupFactory, - @Nonnull ExecutionGraphInfoStore executionGraphInfoStore, - @Nonnull FatalErrorHandler fatalErrorHandler, - @Nonnull HistoryServerArchivist historyServerArchivist, + private PartialDispatcherServicesWithJobPersistenceComponents( + Configuration configuration, + HighAvailabilityServices highAvailabilityServices, + GatewayRetriever resourceManagerGatewayRetriever, + BlobServer blobServer, + HeartbeatServices heartbeatServices, + JobManagerMetricGroupFactory jobManagerMetricGroupFactory, + ExecutionGraphInfoStore executionGraphInfoStore, + FatalErrorHandler fatalErrorHandler, + HistoryServerArchivist historyServerArchivist, @Nullable String metricQueryServiceAddress, - @Nonnull Executor ioExecutor, - @Nonnull DispatcherOperationCaches operationCaches, - @Nonnull JobGraphWriter jobGraphWriter) { + Executor ioExecutor, + DispatcherOperationCaches operationCaches, + JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore) { super( configuration, highAvailabilityServices, @@ -65,16 +68,22 @@ private PartialDispatcherServicesWithJobGraphStore( ioExecutor, operationCaches); this.jobGraphWriter = jobGraphWriter; + this.jobResultStore = jobResultStore; } - @Nonnull public JobGraphWriter getJobGraphWriter() { return jobGraphWriter; } - public static PartialDispatcherServicesWithJobGraphStore from( - PartialDispatcherServices partialDispatcherServices, JobGraphWriter jobGraphWriter) { - return new PartialDispatcherServicesWithJobGraphStore( + public JobResultStore getJobResultStore() { + return jobResultStore; + } + + public static PartialDispatcherServicesWithJobPersistenceComponents from( + PartialDispatcherServices partialDispatcherServices, + JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore) { + return new PartialDispatcherServicesWithJobPersistenceComponents( partialDispatcherServices.getConfiguration(), partialDispatcherServices.getHighAvailabilityServices(), partialDispatcherServices.getResourceManagerGatewayRetriever(), @@ -87,6 +96,7 @@ public static PartialDispatcherServicesWithJobGraphStore from( partialDispatcherServices.getMetricQueryServiceAddress(), partialDispatcherServices.getIoExecutor(), partialDispatcherServices.getOperationCaches(), - jobGraphWriter); + jobGraphWriter, + jobResultStore); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java index 55d92efa366ec..e885958740bbb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SessionDispatcherFactory.java @@ -18,7 +18,9 @@ package org.apache.flink.runtime.dispatcher; +import org.apache.flink.runtime.dispatcher.cleanup.CheckpointResourcesCleanupRunnerFactory; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; import java.util.Collection; @@ -32,17 +34,21 @@ public StandaloneDispatcher createDispatcher( RpcService rpcService, DispatcherId fencingToken, Collection recoveredJobs, + Collection recoveredDirtyJobResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, - PartialDispatcherServicesWithJobGraphStore partialDispatcherServicesWithJobGraphStore) + PartialDispatcherServicesWithJobPersistenceComponents + partialDispatcherServicesWithJobPersistenceComponents) throws Exception { // create the default dispatcher return new StandaloneDispatcher( rpcService, fencingToken, recoveredJobs, + recoveredDirtyJobResults, dispatcherBootstrapFactory, DispatcherServices.from( - partialDispatcherServicesWithJobGraphStore, - JobMasterServiceLeadershipRunnerFactory.INSTANCE)); + partialDispatcherServicesWithJobPersistenceComponents, + JobMasterServiceLeadershipRunnerFactory.INSTANCE, + CheckpointResourcesCleanupRunnerFactory.INSTANCE)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobJobGraphStore.java index f2c450abf868d..b063b4bcad505 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobJobGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/SingleJobJobGraphStore.java @@ -64,16 +64,6 @@ public void putJobGraph(JobGraph jobGraph) throws Exception { } } - @Override - public void removeJobGraph(JobID jobId) { - // ignore - } - - @Override - public void releaseJobGraph(JobID jobId) { - // ignore - } - @Override public Collection getJobIds() { return Collections.singleton(jobGraph.getJobID()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java index 1763bb75fc494..3bb10734ed8ec 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/StandaloneDispatcher.java @@ -20,6 +20,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobMaster; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; import java.util.Collection; @@ -34,6 +35,7 @@ public StandaloneDispatcher( RpcService rpcService, DispatcherId fencingToken, Collection recoveredJobs, + Collection recoveredDirtyJobResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, DispatcherServices dispatcherServices) throws Exception { @@ -41,6 +43,7 @@ public StandaloneDispatcher( rpcService, fencingToken, recoveredJobs, + recoveredDirtyJobResults, dispatcherBootstrapFactory, dispatcherServices); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java new file mode 100644 index 0000000000000..86b707d09ce99 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.CheckpointsCleaner; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStoreUtils; +import org.apache.flink.runtime.dispatcher.UnavailableDispatcherOperationException; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobManagerRunnerResult; +import org.apache.flink.runtime.jobmaster.JobMaster; +import org.apache.flink.runtime.jobmaster.JobMasterGateway; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.messages.webmonitor.JobDetails; +import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.runtime.state.SharedStateRegistryFactory; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.concurrent.FutureUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +/** + * {@code CheckpointResourcesCleanupRunner} implements {@link JobManagerRunner} in a way, that only + * the checkpoint-related resources are instantiated. It triggers any job-specific cleanup that's + * usually performed by the {@link JobMaster} without rebuilding the corresponding {@link + * org.apache.flink.runtime.executiongraph.ExecutionGraph}. + */ +public class CheckpointResourcesCleanupRunner implements JobManagerRunner { + + private static final Logger LOG = + LoggerFactory.getLogger(CheckpointResourcesCleanupRunner.class); + + private final JobResult jobResult; + private final CheckpointRecoveryFactory checkpointRecoveryFactory; + private final CheckpointsCleaner checkpointsCleaner; + private final SharedStateRegistryFactory sharedStateRegistryFactory; + private final Configuration jobManagerConfiguration; + private final Executor cleanupExecutor; + + private final long initializationTimestamp; + + // we have to have two separate futures because closeAsync relies on the completion of + // getResultFuture which is always already completed but the cleanupFuture is only + // instantiated when calling start + private CompletableFuture cleanupFuture; + private final CompletableFuture closeFuture = new CompletableFuture<>(); + + private CompletedCheckpointStore completedCheckpointStore; + private CheckpointIDCounter checkpointIDCounter; + + public CheckpointResourcesCleanupRunner( + JobResult jobResult, + CheckpointRecoveryFactory checkpointRecoveryFactory, + CheckpointsCleaner checkpointsCleaner, + SharedStateRegistryFactory sharedStateRegistryFactory, + Configuration jobManagerConfiguration, + Executor cleanupExecutor, + long initializationTimestamp) { + this.jobResult = Preconditions.checkNotNull(jobResult); + this.checkpointRecoveryFactory = Preconditions.checkNotNull(checkpointRecoveryFactory); + this.checkpointsCleaner = Preconditions.checkNotNull(checkpointsCleaner); + this.sharedStateRegistryFactory = Preconditions.checkNotNull(sharedStateRegistryFactory); + this.jobManagerConfiguration = Preconditions.checkNotNull(jobManagerConfiguration); + this.cleanupExecutor = Preconditions.checkNotNull(cleanupExecutor); + this.initializationTimestamp = initializationTimestamp; + } + + @Override + public CompletableFuture closeAsync() { + return closeFuture; + } + + @Override + public void start() throws Exception { + cleanupFuture = + CompletableFuture.runAsync(this::initializeAccessingComponents, cleanupExecutor) + .thenApply( + result -> { + Exception exception = null; + try { + completedCheckpointStore.shutdown( + getJobStatus(), checkpointsCleaner); + } catch (Exception e) { + exception = e; + } + + try { + checkpointIDCounter.shutdown(getJobStatus()); + } catch (Exception e) { + exception = ExceptionUtils.firstOrSuppressed(e, exception); + } + + if (exception != null) { + throw new CompletionException(exception); + } + + return null; + }); + + FutureUtils.forward(cleanupFuture, closeFuture); + } + + private void initializeAccessingComponents() { + initializeCompletedCheckpointStore(); + initializeCheckpointIDCounter(); + } + + private void initializeCompletedCheckpointStore() { + try { + this.completedCheckpointStore = + checkpointRecoveryFactory.createRecoveredCompletedCheckpointStore( + getJobID(), + CompletedCheckpointStoreUtils.getMaximumNumberOfRetainedCheckpoints( + jobManagerConfiguration, LOG), + sharedStateRegistryFactory, + cleanupExecutor); + } catch (Exception e) { + throw new CompletionException( + "Error occurred while initializing the CompletedCheckpointStore access.", e); + } + } + + private void initializeCheckpointIDCounter() { + try { + this.checkpointIDCounter = + checkpointRecoveryFactory.createCheckpointIDCounter(getJobID()); + } catch (Exception e) { + throw new CompletionException( + "Error occurred while initializing the CheckpointIDCounter access.", e); + } + } + + @Override + public CompletableFuture getJobMasterGateway() { + return FutureUtils.completedExceptionally( + new UnavailableDispatcherOperationException( + "Unable to get JobMasterGateway for job in cleanup phase. The requested operation is not available in that stage.")); + } + + @Override + public CompletableFuture getResultFuture() { + return CompletableFuture.completedFuture( + JobManagerRunnerResult.forSuccess(createExecutionGraphInfoFromJobResult())); + } + + @Override + public JobID getJobID() { + return jobResult.getJobId(); + } + + @Override + public CompletableFuture cancel(Time timeout) { + Preconditions.checkState( + cleanupFuture != null, + "The CheckpointResourcesCleanupRunner was not started, yet."); + if (cleanupFuture.cancel(true)) { + return CompletableFuture.completedFuture(Acknowledge.get()); + } + + return FutureUtils.completedExceptionally( + new FlinkException("Cleanup task couldn't be cancelled.")); + } + + @Override + public CompletableFuture requestJobStatus(Time timeout) { + return CompletableFuture.completedFuture(getJobStatus()); + } + + @Override + public CompletableFuture requestJobDetails(Time timeout) { + return requestJob(timeout) + .thenApply( + executionGraphInfo -> + JobDetails.createDetailsForJob( + executionGraphInfo.getArchivedExecutionGraph())); + } + + @Override + public CompletableFuture requestJob(Time timeout) { + return CompletableFuture.completedFuture(createExecutionGraphInfoFromJobResult()); + } + + @Override + public boolean isInitialized() { + return true; + } + + private ExecutionGraphInfo createExecutionGraphInfoFromJobResult() { + return generateExecutionGraphInfo(jobResult, initializationTimestamp); + } + + private JobStatus getJobStatus() { + return getJobStatus(jobResult); + } + + private static JobStatus getJobStatus(JobResult jobResult) { + return jobResult.getApplicationStatus().deriveJobStatus(); + } + + private static ExecutionGraphInfo generateExecutionGraphInfo( + JobResult jobResult, long initializationTimestamp) { + return new ExecutionGraphInfo( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( + jobResult.getJobId(), + "unknown", + getJobStatus(jobResult), + jobResult.getSerializedThrowable().orElse(null), + null, + initializationTimestamp)); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerFactory.java new file mode 100644 index 0000000000000..302d318336b92 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerFactory.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.CheckpointsCleaner; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.state.SharedStateRegistry; + +import java.util.concurrent.Executor; + +/** + * {@code CheckpointResourcesCleanupRunnerFactory} implements {@link CleanupRunnerFactory} providing + * a factory method for creating {@link CheckpointResourcesCleanupRunner} instances. + */ +public enum CheckpointResourcesCleanupRunnerFactory implements CleanupRunnerFactory { + INSTANCE; + + @Override + public CheckpointResourcesCleanupRunner create( + JobResult jobResult, + CheckpointRecoveryFactory checkpointRecoveryFactory, + Configuration configuration, + Executor cleanupExecutor) { + return new CheckpointResourcesCleanupRunner( + jobResult, + checkpointRecoveryFactory, + new CheckpointsCleaner(), + SharedStateRegistry.DEFAULT_FACTORY, + configuration, + cleanupExecutor, + System.currentTimeMillis()); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRunnerFactory.java new file mode 100644 index 0000000000000..d9d8ebaf2cd0d --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRunnerFactory.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.JobResult; + +import java.util.concurrent.Executor; + +/** + * {@code CleanupRunnerFactory} provides a factory method for creating {@link + * CheckpointResourcesCleanupRunner} instances. + */ +@FunctionalInterface +public interface CleanupRunnerFactory { + JobManagerRunner create( + JobResult jobResult, + CheckpointRecoveryFactory checkpointRecoveryFactory, + Configuration configuration, + Executor cleanupExecutor); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultResourceCleaner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultResourceCleaner.java new file mode 100644 index 0000000000000..42bfeba66e892 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultResourceCleaner.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.util.concurrent.FutureUtils; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.function.BiFunction; +import java.util.stream.Collectors; + +/** {@code DefaultResourceCleaner} is the default implementation of {@link ResourceCleaner}. */ +public class DefaultResourceCleaner implements ResourceCleaner { + + private final Collection>> + prioritizedOrderedJobRelatedCleanups = new ArrayList<>(); + private final Collection>> + jobRelatedCleanups = new ArrayList<>(); + private final Executor cleanupExecutor; + + DefaultResourceCleaner(Executor cleanupExecutor) { + this.cleanupExecutor = cleanupExecutor; + } + + DefaultResourceCleaner withPriorityCleanupOf( + BiFunction> cleanupCallback) { + prioritizedOrderedJobRelatedCleanups.add(cleanupCallback); + return this; + } + + DefaultResourceCleaner withCleanupOf( + BiFunction> cleanupCallback) { + jobRelatedCleanups.add(cleanupCallback); + return this; + } + + @Override + public CompletableFuture cleanupAsync(JobID jobId) { + CompletableFuture prioritizedOrderedCleanupFuture = FutureUtils.completedVoidFuture(); + for (BiFunction> prioritizedCleanupFunction : + prioritizedOrderedJobRelatedCleanups) { + prioritizedOrderedCleanupFuture = + prioritizedOrderedCleanupFuture.thenCompose( + ignoredValue -> + prioritizedCleanupFunction.apply(jobId, cleanupExecutor)); + } + + return prioritizedOrderedCleanupFuture.thenCompose( + ignoredValue -> + FutureUtils.completeAll( + jobRelatedCleanups.stream() + .map(c -> c.apply(jobId, cleanupExecutor)) + .collect(Collectors.toList()))); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactory.java new file mode 100644 index 0000000000000..0584a75a6816e --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactory.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.dispatcher.DispatcherServices; +import org.apache.flink.runtime.dispatcher.JobManagerRunnerRegistry; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.jobmanager.JobGraphWriter; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; +import org.apache.flink.util.Preconditions; + +import java.util.Arrays; +import java.util.concurrent.Executor; + +/** + * {@code DispatcherResourceCleanerFactory} instantiates {@link ResourceCleaner} instances that + * clean cleanable resources from the {@link org.apache.flink.runtime.dispatcher.Dispatcher}. + * + *

We need to handle the {@link JobManagerRunnerRegistry} differently due to a dependency between + * closing the {@link org.apache.flink.runtime.jobmaster.JobManagerRunner} and the {@link + * HighAvailabilityServices}. This is fixed in {@code FLINK-24038} using a feature flag to + * enable/disable single leader election for all the {@code JobManager} components. We can remove + * the priority cleanup logic after removing the per-component leader election. + */ +public class DispatcherResourceCleanerFactory implements ResourceCleanerFactory { + + private final Executor cleanupExecutor; + private final JobManagerRunnerRegistry jobManagerRunnerRegistry; + private final JobGraphWriter jobGraphWriter; + private final BlobServer blobServer; + private final HighAvailabilityServices highAvailabilityServices; + private final JobManagerMetricGroup jobManagerMetricGroup; + + public DispatcherResourceCleanerFactory( + JobManagerRunnerRegistry jobManagerRunnerRegistry, + DispatcherServices dispatcherServices) { + this( + dispatcherServices.getIoExecutor(), + jobManagerRunnerRegistry, + dispatcherServices.getJobGraphWriter(), + dispatcherServices.getBlobServer(), + dispatcherServices.getHighAvailabilityServices(), + dispatcherServices.getJobManagerMetricGroup()); + } + + @VisibleForTesting + public DispatcherResourceCleanerFactory( + Executor cleanupExecutor, + JobManagerRunnerRegistry jobManagerRunnerRegistry, + JobGraphWriter jobGraphWriter, + BlobServer blobServer, + HighAvailabilityServices highAvailabilityServices, + JobManagerMetricGroup jobManagerMetricGroup) { + this.cleanupExecutor = Preconditions.checkNotNull(cleanupExecutor); + this.jobManagerRunnerRegistry = Preconditions.checkNotNull(jobManagerRunnerRegistry); + this.jobGraphWriter = Preconditions.checkNotNull(jobGraphWriter); + this.blobServer = Preconditions.checkNotNull(blobServer); + this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityServices); + this.jobManagerMetricGroup = Preconditions.checkNotNull(jobManagerMetricGroup); + } + + public ResourceCleaner createLocalResourceCleaner() { + final DefaultResourceCleaner resourceCleaner = new DefaultResourceCleaner(cleanupExecutor); + resourceCleaner.withPriorityCleanupOf(jobManagerRunnerRegistry::localCleanupAsync); + for (LocallyCleanableResource locallyCleanableResource : + Arrays.asList( + jobGraphWriter, + blobServer, + highAvailabilityServices, + jobManagerMetricGroup)) { + resourceCleaner.withCleanupOf(locallyCleanableResource::localCleanupAsync); + } + + return resourceCleaner; + } + + public ResourceCleaner createGlobalResourceCleaner() { + final DefaultResourceCleaner resourceCleaner = new DefaultResourceCleaner(cleanupExecutor); + resourceCleaner.withPriorityCleanupOf(jobManagerRunnerRegistry::globalCleanupAsync); + for (GloballyCleanableResource globallyCleanableResource : + Arrays.asList( + jobGraphWriter, + blobServer, + highAvailabilityServices, + jobManagerMetricGroup)) { + resourceCleaner.withCleanupOf(globallyCleanableResource::globalCleanupAsync); + } + + return resourceCleaner; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/GloballyCleanableResource.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/GloballyCleanableResource.java new file mode 100644 index 0000000000000..57ba065f84ace --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/GloballyCleanableResource.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +/** + * {@code GloballyCleanableResource} is supposed to be used by any class that provides artifacts for + * a given job that can be cleaned up globally. + * + * @see LocallyCleanableResource + */ +@FunctionalInterface +public interface GloballyCleanableResource { + + void globalCleanup(JobID jobId) throws Throwable; + + default CompletableFuture globalCleanupAsync(JobID jobId, Executor cleanupExecutor) { + return CompletableFuture.runAsync( + () -> { + try { + globalCleanup(jobId); + } catch (Throwable t) { + throw new CompletionException("Asynchronous global cleanup failed", t); + } + }, + cleanupExecutor); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/LocallyCleanableResource.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/LocallyCleanableResource.java new file mode 100644 index 0000000000000..fd2b98caed62b --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/LocallyCleanableResource.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +/** + * {@code LocallyCleanableResource} is supposed to be used by any class that provides artifacts for + * a given job that can be cleaned up locally. + * + * @see GloballyCleanableResource + */ +@FunctionalInterface +public interface LocallyCleanableResource { + + void localCleanup(JobID jobId) throws Throwable; + + default CompletableFuture localCleanupAsync(JobID jobId, Executor cleanupExecutor) { + return CompletableFuture.runAsync( + () -> { + try { + localCleanup(jobId); + } catch (Throwable t) { + throw new CompletionException("Asynchronous local cleanup failed", t); + } + }, + cleanupExecutor); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ResourceCleaner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ResourceCleaner.java new file mode 100644 index 0000000000000..9a43dd59a7e16 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ResourceCleaner.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; + +import java.util.concurrent.CompletableFuture; + +/** {@code ResourceCleaner} executes instances on the given {@code JobID}. */ +@FunctionalInterface +public interface ResourceCleaner { + + CompletableFuture cleanupAsync(JobID jobId); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ResourceCleanerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ResourceCleanerFactory.java new file mode 100644 index 0000000000000..91584dc0a2e31 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ResourceCleanerFactory.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; + +import java.util.concurrent.Executor; + +/** + * {@code ResourceCleanerFactory} provides methods to create {@link ResourceCleaner} for local and + * global cleanup. + * + * @see GloballyCleanableResource + * @see LocallyCleanableResource + */ +public interface ResourceCleanerFactory { + + /** + * Creates {@link ResourceCleaner} that initiates {@link + * LocallyCleanableResource#localCleanupAsync(JobID, Executor)} calls. + */ + ResourceCleaner createLocalResourceCleaner(); + + /** + * Creates {@link ResourceCleaner} that initiates {@link + * GloballyCleanableResource#globalCleanupAsync(JobID, Executor)} calls. + */ + ResourceCleaner createGlobalResourceCleaner(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java index b5bf60f19a7bb..faf161e1ba126 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/AbstractDispatcherLeaderProcess.java @@ -24,8 +24,10 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.DispatcherId; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphWriter; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.webmonitor.RestfulGateway; import org.apache.flink.util.AutoCloseableAsync; @@ -176,16 +178,14 @@ private void handleUnexpectedDispatcherServiceTermination( createdDispatcherService .getTerminationFuture() .whenComplete( - (ignored, throwable) -> { - runIfStateIs( - State.RUNNING, - () -> { - handleError( - new FlinkException( - "Unexpected termination of DispatcherService.", - throwable)); - }); - }); + (ignored, throwable) -> + runIfStateIs( + State.RUNNING, + () -> + handleError( + new FlinkException( + "Unexpected termination of DispatcherService.", + throwable)))); } final Optional supplyUnsynchronizedIfRunning(Supplier supplier) { @@ -257,9 +257,11 @@ protected enum State { /** Factory for {@link DispatcherGatewayService}. */ public interface DispatcherGatewayServiceFactory { DispatcherGatewayService create( - DispatcherId fencingToken, + DispatcherId dispatcherId, Collection recoveredJobs, - JobGraphWriter jobGraphWriter); + Collection recoveredDirtyJobResults, + JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore); } /** An accessor of the {@link DispatcherGateway}. */ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java index e521c4a75e635..40498fee7df53 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherGatewayServiceFactory.java @@ -23,9 +23,11 @@ import org.apache.flink.runtime.dispatcher.DispatcherId; import org.apache.flink.runtime.dispatcher.NoOpDispatcherBootstrap; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobPersistenceComponents; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphWriter; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.FlinkRuntimeException; @@ -54,7 +56,9 @@ class DefaultDispatcherGatewayServiceFactory public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( DispatcherId fencingToken, Collection recoveredJobs, - JobGraphWriter jobGraphWriter) { + Collection recoveredDirtyJobResults, + JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore) { final Dispatcher dispatcher; try { @@ -63,10 +67,11 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( rpcService, fencingToken, recoveredJobs, + recoveredDirtyJobResults, (dispatcherGateway, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(), - PartialDispatcherServicesWithJobGraphStore.from( - partialDispatcherServices, jobGraphWriter)); + PartialDispatcherServicesWithJobPersistenceComponents.from( + partialDispatcherServices, jobGraphWriter, jobResultStore)); } catch (Exception e) { throw new FlinkRuntimeException("Could not create the Dispatcher rpc endpoint.", e); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java index 9d5c4d62dd4ff..2050271db1634 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DefaultDispatcherRunnerFactory.java @@ -21,7 +21,7 @@ import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.entrypoint.component.JobGraphRetriever; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -44,7 +44,7 @@ public DefaultDispatcherRunnerFactory( public DispatcherRunner createDispatcherRunner( LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler, - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices) @@ -52,7 +52,7 @@ public DispatcherRunner createDispatcherRunner( final DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory = dispatcherLeaderProcessFactoryFactory.createFactory( - jobGraphStoreFactory, + jobPersistenceComponentFactory, ioExecutor, rpcService, partialDispatcherServices, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java index 6792fa32ecccc..222c349870100 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherLeaderProcessFactoryFactory.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -31,7 +31,7 @@ public interface DispatcherLeaderProcessFactoryFactory { DispatcherLeaderProcessFactory createFactory( - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactory.java index a03be4cf42b48..bebafa2bbf0c2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/DispatcherRunnerFactory.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -32,7 +32,7 @@ public interface DispatcherRunnerFactory { DispatcherRunner createDispatcherRunner( LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler, - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java index f50b396ba0a36..6f56e2cf7db48 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcess.java @@ -19,11 +19,16 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.runtime.dispatcher.DispatcherId; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.ThrowingJobGraphWriter; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.util.CollectionUtil; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; -import java.util.Collections; import java.util.UUID; /** {@link DispatcherLeaderProcess} implementation for the per-job mode. */ @@ -31,16 +36,23 @@ public class JobDispatcherLeaderProcess extends AbstractDispatcherLeaderProcess private final DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory; - private final JobGraph jobGraph; + @Nullable private final JobGraph jobGraph; + @Nullable private final JobResult recoveredDirtyJobResult; + + private final JobResultStore jobResultStore; JobDispatcherLeaderProcess( UUID leaderSessionId, DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, - JobGraph jobGraph, + @Nullable JobGraph jobGraph, + @Nullable JobResult recoveredDirtyJobResult, + JobResultStore jobResultStore, FatalErrorHandler fatalErrorHandler) { super(leaderSessionId, fatalErrorHandler); - this.jobGraph = jobGraph; this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; + this.jobGraph = jobGraph; + this.recoveredDirtyJobResult = recoveredDirtyJobResult; + this.jobResultStore = Preconditions.checkNotNull(jobResultStore); } @Override @@ -48,8 +60,10 @@ protected void onStart() { final DispatcherGatewayService dispatcherService = dispatcherGatewayServiceFactory.create( DispatcherId.fromUuid(getLeaderSessionId()), - Collections.singleton(jobGraph), - ThrowingJobGraphWriter.INSTANCE); + CollectionUtil.ofNullable(jobGraph), + CollectionUtil.ofNullable(recoveredDirtyJobResult), + ThrowingJobGraphWriter.INSTANCE, + jobResultStore); completeDispatcherSetup(dispatcherService); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java index 2081f60c69ea5..1716a1e961a19 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactory.java @@ -18,33 +18,61 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; import java.util.UUID; /** Factory for the {@link JobDispatcherLeaderProcess}. */ public class JobDispatcherLeaderProcessFactory implements DispatcherLeaderProcessFactory { + private final AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory; - private final JobGraph jobGraph; + @Nullable private final JobGraph jobGraph; + @Nullable private final JobResult recoveredDirtyJobResult; + + private final JobResultStore jobResultStore; private final FatalErrorHandler fatalErrorHandler; JobDispatcherLeaderProcessFactory( AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, - JobGraph jobGraph, + @Nullable JobGraph jobGraph, + @Nullable JobResult recoveredDirtyJobResult, + JobResultStore jobResultStore, FatalErrorHandler fatalErrorHandler) { this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; this.jobGraph = jobGraph; + this.recoveredDirtyJobResult = recoveredDirtyJobResult; + this.jobResultStore = Preconditions.checkNotNull(jobResultStore); this.fatalErrorHandler = fatalErrorHandler; } @Override public DispatcherLeaderProcess create(UUID leaderSessionID) { return new JobDispatcherLeaderProcess( - leaderSessionID, dispatcherGatewayServiceFactory, jobGraph, fatalErrorHandler); + leaderSessionID, + dispatcherGatewayServiceFactory, + jobGraph, + recoveredDirtyJobResult, + jobResultStore, + fatalErrorHandler); + } + + @Nullable + JobGraph getJobGraph() { + return this.jobGraph; + } + + @Nullable + JobResult getRecoveredDirtyJobResult() { + return this.recoveredDirtyJobResult; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java index 7be59105478be..2ebc3bce8fcbd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactory.java @@ -18,31 +18,48 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.dispatcher.JobDispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; import org.apache.flink.runtime.entrypoint.component.JobGraphRetriever; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.FlinkException; import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collection; +import java.util.Optional; +import java.util.Set; import java.util.concurrent.Executor; +import java.util.stream.Collectors; /** Factory for the {@link JobDispatcherLeaderProcessFactory}. */ public class JobDispatcherLeaderProcessFactoryFactory implements DispatcherLeaderProcessFactoryFactory { + private static final Logger LOG = + LoggerFactory.getLogger(JobDispatcherLeaderProcessFactoryFactory.class); + private final JobGraphRetriever jobGraphRetriever; - private JobDispatcherLeaderProcessFactoryFactory(JobGraphRetriever jobGraphRetriever) { + @VisibleForTesting + JobDispatcherLeaderProcessFactoryFactory(JobGraphRetriever jobGraphRetriever) { this.jobGraphRetriever = jobGraphRetriever; } @Override - public DispatcherLeaderProcessFactory createFactory( - JobGraphStoreFactory jobGraphStoreFactory, + public JobDispatcherLeaderProcessFactory createFactory( + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices, @@ -52,22 +69,70 @@ public DispatcherLeaderProcessFactory createFactory( try { jobGraph = - jobGraphRetriever.retrieveJobGraph( - partialDispatcherServices.getConfiguration()); + Preconditions.checkNotNull( + jobGraphRetriever.retrieveJobGraph( + partialDispatcherServices.getConfiguration())); } catch (FlinkException e) { throw new FlinkRuntimeException("Could not retrieve the JobGraph.", e); } + final JobResultStore jobResultStore = jobPersistenceComponentFactory.createJobResultStore(); + final Collection recoveredDirtyJobResults = getDirtyJobResults(jobResultStore); + + final Optional maybeRecoveredDirtyJobResult = + extractDirtyJobResult(recoveredDirtyJobResults, jobGraph); + final Optional maybeJobGraph = + getJobGraphBasedOnDirtyJobResults(jobGraph, recoveredDirtyJobResults); + final DefaultDispatcherGatewayServiceFactory defaultDispatcherServiceFactory = new DefaultDispatcherGatewayServiceFactory( JobDispatcherFactory.INSTANCE, rpcService, partialDispatcherServices); return new JobDispatcherLeaderProcessFactory( - defaultDispatcherServiceFactory, jobGraph, fatalErrorHandler); + defaultDispatcherServiceFactory, + maybeJobGraph.orElse(null), + maybeRecoveredDirtyJobResult.orElse(null), + jobResultStore, + fatalErrorHandler); } public static JobDispatcherLeaderProcessFactoryFactory create( JobGraphRetriever jobGraphRetriever) { return new JobDispatcherLeaderProcessFactoryFactory(jobGraphRetriever); } + + private static Collection getDirtyJobResults(JobResultStore jobResultStore) { + try { + return jobResultStore.getDirtyResults(); + } catch (IOException e) { + throw new FlinkRuntimeException( + "Could not retrieve the JobResults of dirty jobs from the underlying JobResultStore.", + e); + } + } + + private static Optional extractDirtyJobResult( + Collection dirtyJobResults, JobGraph jobGraph) { + Optional actualDirtyJobResult = Optional.empty(); + for (JobResult dirtyJobResult : dirtyJobResults) { + if (dirtyJobResult.getJobId().equals(jobGraph.getJobID())) { + actualDirtyJobResult = Optional.of(dirtyJobResult); + } else { + LOG.warn( + "Unexpected dirty JobResultStore entry: Job '{}' is listed as dirty, isn't part of this single-job cluster, though.", + dirtyJobResult.getJobId()); + } + } + + return actualDirtyJobResult; + } + + private static Optional getJobGraphBasedOnDirtyJobResults( + JobGraph jobGraph, Collection dirtyJobResults) { + final Set jobIdsOfFinishedJobs = + dirtyJobResults.stream().map(JobResult::getJobId).collect(Collectors.toSet()); + return jobIdsOfFinishedJobs.contains(jobGraph.getJobID()) + ? Optional.empty() + : Optional.of(jobGraph); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java index efc6b2098a997..6f9d0bf2fd377 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java @@ -23,8 +23,10 @@ import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.DispatcherId; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.util.ExceptionUtils; @@ -33,14 +35,17 @@ import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.function.FunctionUtils; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Optional; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.Executor; +import java.util.stream.Collectors; /** * Process which encapsulates the job recovery logic and life cycle management of a {@link @@ -53,6 +58,8 @@ public class SessionDispatcherLeaderProcess extends AbstractDispatcherLeaderProc private final JobGraphStore jobGraphStore; + private final JobResultStore jobResultStore; + private final Executor ioExecutor; private CompletableFuture onGoingRecoveryOperation = FutureUtils.completedVoidFuture(); @@ -61,12 +68,14 @@ private SessionDispatcherLeaderProcess( UUID leaderSessionId, DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, JobGraphStore jobGraphStore, + JobResultStore jobResultStore, Executor ioExecutor, FatalErrorHandler fatalErrorHandler) { super(leaderSessionId, fatalErrorHandler); this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; this.jobGraphStore = jobGraphStore; + this.jobResultStore = jobResultStore; this.ioExecutor = ioExecutor; } @@ -75,9 +84,7 @@ protected void onStart() { startServices(); onGoingRecoveryOperation = - recoverJobsAsync() - .thenAccept(this::createDispatcherIfRunning) - .handle(this::onErrorIfRunning); + createDispatcherBasedOnRecoveredJobGraphsAndRecoveredDirtyJobResults(); } private void startServices() { @@ -92,34 +99,56 @@ private void startServices() { } } - private void createDispatcherIfRunning(Collection jobGraphs) { - runIfStateIs(State.RUNNING, () -> createDispatcher(jobGraphs)); + private void createDispatcherIfRunning( + Collection jobGraphs, Collection recoveredDirtyJobResults) { + runIfStateIs(State.RUNNING, () -> createDispatcher(jobGraphs, recoveredDirtyJobResults)); } - private void createDispatcher(Collection jobGraphs) { + private void createDispatcher( + Collection jobGraphs, Collection recoveredDirtyJobResults) { final DispatcherGatewayService dispatcherService = dispatcherGatewayServiceFactory.create( - DispatcherId.fromUuid(getLeaderSessionId()), jobGraphs, jobGraphStore); + DispatcherId.fromUuid(getLeaderSessionId()), + jobGraphs, + recoveredDirtyJobResults, + jobGraphStore, + jobResultStore); completeDispatcherSetup(dispatcherService); } - private CompletableFuture> recoverJobsAsync() { - return CompletableFuture.supplyAsync(this::recoverJobsIfRunning, ioExecutor); + private CompletableFuture + createDispatcherBasedOnRecoveredJobGraphsAndRecoveredDirtyJobResults() { + final CompletableFuture> dirtyJobsFuture = + CompletableFuture.supplyAsync(this::getDirtyJobResultsIfRunning, ioExecutor); + + return dirtyJobsFuture + .thenApplyAsync( + dirtyJobs -> + this.recoverJobsIfRunning( + dirtyJobs.stream() + .map(JobResult::getJobId) + .collect(Collectors.toSet())), + ioExecutor) + .thenAcceptBoth(dirtyJobsFuture, this::createDispatcherIfRunning) + .handle(this::onErrorIfRunning); } - private Collection recoverJobsIfRunning() { - return supplyUnsynchronizedIfRunning(this::recoverJobs).orElse(Collections.emptyList()); + private Collection recoverJobsIfRunning(Set recoveredDirtyJobResults) { + return supplyUnsynchronizedIfRunning(() -> recoverJobs(recoveredDirtyJobResults)) + .orElse(Collections.emptyList()); } - private Collection recoverJobs() { - log.info("Recover all persisted job graphs."); + private Collection recoverJobs(Set recoveredDirtyJobResults) { + log.info("Recover all persisted job graphs that are not finished, yet."); final Collection jobIds = getJobIds(); final Collection recoveredJobGraphs = new ArrayList<>(); for (JobID jobId : jobIds) { - recoveredJobGraphs.add(recoverJob(jobId)); + if (!recoveredDirtyJobResults.contains(jobId)) { + recoveredJobGraphs.add(recoverJob(jobId)); + } } log.info("Successfully recovered {} persisted job graphs.", recoveredJobGraphs.size()); @@ -145,6 +174,21 @@ private JobGraph recoverJob(JobID jobId) { } } + private Collection getDirtyJobResultsIfRunning() { + return supplyUnsynchronizedIfRunning(this::getDirtyJobResults) + .orElse(Collections.emptyList()); + } + + private Collection getDirtyJobResults() { + try { + return jobResultStore.getDirtyResults(); + } catch (IOException e) { + throw new FlinkRuntimeException( + "Could not retrieve JobResults of globally-terminated jobs from JobResultStore", + e); + } + } + @Override protected CompletableFuture onClose() { return CompletableFuture.runAsync(this::stopServices, ioExecutor); @@ -261,9 +305,15 @@ public static SessionDispatcherLeaderProcess create( UUID leaderSessionId, DispatcherGatewayServiceFactory dispatcherFactory, JobGraphStore jobGraphStore, + JobResultStore jobResultStore, Executor ioExecutor, FatalErrorHandler fatalErrorHandler) { return new SessionDispatcherLeaderProcess( - leaderSessionId, dispatcherFactory, jobGraphStore, ioExecutor, fatalErrorHandler); + leaderSessionId, + dispatcherFactory, + jobGraphStore, + jobResultStore, + ioExecutor, + fatalErrorHandler); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java index 95cf4e49e10ee..8c08bb0d0b39c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactory.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.annotation.Internal; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; import java.util.UUID; @@ -31,18 +31,18 @@ public class SessionDispatcherLeaderProcessFactory implements DispatcherLeaderPr private final AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory; - private final JobGraphStoreFactory jobGraphStoreFactory; + private final JobPersistenceComponentFactory jobPersistenceComponentFactory; private final Executor ioExecutor; private final FatalErrorHandler fatalErrorHandler; public SessionDispatcherLeaderProcessFactory( AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, FatalErrorHandler fatalErrorHandler) { this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; - this.jobGraphStoreFactory = jobGraphStoreFactory; + this.jobPersistenceComponentFactory = jobPersistenceComponentFactory; this.ioExecutor = ioExecutor; this.fatalErrorHandler = fatalErrorHandler; } @@ -52,7 +52,8 @@ public DispatcherLeaderProcess create(UUID leaderSessionID) { return SessionDispatcherLeaderProcess.create( leaderSessionID, dispatcherGatewayServiceFactory, - jobGraphStoreFactory.create(), + jobPersistenceComponentFactory.createJobGraphStore(), + jobPersistenceComponentFactory.createJobResultStore(), ioExecutor, fatalErrorHandler); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java index 29789650ba4f3..ee2fb081647fe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessFactoryFactory.java @@ -20,7 +20,7 @@ import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -38,7 +38,7 @@ private SessionDispatcherLeaderProcessFactoryFactory(DispatcherFactory dispatche @Override public DispatcherLeaderProcessFactory createFactory( - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices, @@ -50,7 +50,7 @@ public DispatcherLeaderProcessFactory createFactory( return new SessionDispatcherLeaderProcessFactory( dispatcherGatewayServiceFactory, - jobGraphStoreFactory, + jobPersistenceComponentFactory, ioExecutor, fatalErrorHandler); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java index 0d514f5183b18..f235caa554dc0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/entrypoint/component/DefaultDispatcherResourceManagerComponentFactory.java @@ -36,7 +36,7 @@ import org.apache.flink.runtime.entrypoint.ClusterInformation; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.jobmanager.HaServicesJobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.HaServicesJobPersistenceComponentFactory; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; @@ -221,7 +221,7 @@ public DispatcherResourceManagerComponent create( dispatcherRunnerFactory.createDispatcherRunner( highAvailabilityServices.getDispatcherLeaderElectionService(), fatalErrorHandler, - new HaServicesJobGraphStoreFactory(highAvailabilityServices), + new HaServicesJobPersistenceComponentFactory(highAvailabilityServices), ioExecutor, rpcService, partialDispatcherServices); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java index a36cdeb35075b..3d594edd5f9fe 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraph.java @@ -334,7 +334,7 @@ public static ArchivedExecutionGraph createFrom( * Create a sparse ArchivedExecutionGraph for a job while it is still initializing. Most fields * will be empty, only job status and error-related fields are set. */ - public static ArchivedExecutionGraph createFromInitializingJob( + public static ArchivedExecutionGraph createSparseArchivedExecutionGraph( JobID jobId, String jobName, JobStatus jobStatus, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java index da8fb83266f5e..d1c1926a6d1ff 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractHaServices.java @@ -47,7 +47,7 @@ * resources. * *

The abstract class is also responsible for determining which component service should be - * reused. For example, {@link #runningJobsRegistry} is created once and could be reused many times. + * reused. For example, {@link #jobResultStore} is created once and could be reused many times. */ public abstract class AbstractHaServices implements HighAvailabilityServices { @@ -62,15 +62,18 @@ public abstract class AbstractHaServices implements HighAvailabilityServices { /** Store for arbitrary blobs. */ private final BlobStoreService blobStoreService; - /** The distributed storage based running jobs registry. */ - private RunningJobsRegistry runningJobsRegistry; + private final JobResultStore jobResultStore; - public AbstractHaServices( - Configuration config, Executor ioExecutor, BlobStoreService blobStoreService) { + protected AbstractHaServices( + Configuration config, + Executor ioExecutor, + BlobStoreService blobStoreService, + JobResultStore jobResultStore) { this.configuration = checkNotNull(config); this.ioExecutor = checkNotNull(ioExecutor); this.blobStoreService = checkNotNull(blobStoreService); + this.jobResultStore = checkNotNull(jobResultStore); } @Override @@ -130,11 +133,8 @@ public JobGraphStore getJobGraphStore() throws Exception { } @Override - public RunningJobsRegistry getRunningJobsRegistry() { - if (runningJobsRegistry == null) { - this.runningJobsRegistry = createRunningJobsRegistry(); - } - return runningJobsRegistry; + public JobResultStore getJobResultStore() throws Exception { + return jobResultStore; } @Override @@ -206,7 +206,7 @@ public void closeAndCleanupAllData() throws Exception { } @Override - public void cleanupJobData(JobID jobID) throws Exception { + public void globalCleanup(JobID jobID) throws Exception { logger.info("Clean up the high availability data for job {}.", jobID); internalCleanupJobData(jobID); logger.info("Finished cleaning up the high availability data for job {}.", jobID); @@ -243,13 +243,6 @@ public void cleanupJobData(JobID jobID) throws Exception { */ protected abstract JobGraphStore createJobGraphStore() throws Exception; - /** - * Create the registry that holds information about whether jobs are currently running. - * - * @return Running job registry to retrieve running jobs - */ - protected abstract RunningJobsRegistry createRunningJobsRegistry(); - /** * Closes the components which is used for external operations(e.g. Zookeeper Client, Kubernetes * Client). diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java index 47a73276703d0..672973239ca72 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java @@ -21,6 +21,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.blob.BlobStore; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.dispatcher.cleanup.GloballyCleanableResource; +import org.apache.flink.runtime.dispatcher.cleanup.LocallyCleanableResource; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -43,7 +45,10 @@ *

  • Naming of RPC endpoints * */ -public interface HighAvailabilityServices extends ClientHighAvailabilityServices { +public interface HighAvailabilityServices + extends ClientHighAvailabilityServices, + LocallyCleanableResource, + GloballyCleanableResource { // ------------------------------------------------------------------------ // Constants @@ -169,11 +174,12 @@ default LeaderElectionService getWebMonitorLeaderElectionService() { JobGraphStore getJobGraphStore() throws Exception; /** - * Gets the registry that holds information about whether jobs are currently running. + * Gets the store that holds information about the state of finished jobs. * - * @return Running job registry to retrieve running jobs + * @return Store of finished job results + * @throws Exception if job result store could not be created */ - RunningJobsRegistry getRunningJobsRegistry() throws Exception; + JobResultStore getJobResultStore() throws Exception; /** * Creates the BLOB store in which BLOBs are stored in a highly-available fashion. @@ -238,11 +244,9 @@ default LeaderRetrievalService getClusterRestEndpointLeaderRetriever() { */ void closeAndCleanupAllData() throws Exception; - /** - * Deletes all data for specified job stored by these services in external stores. - * - * @param jobID The identifier of the job to cleanup. - * @throws Exception Thrown, if an exception occurred while cleaning data stored by them. - */ - default void cleanupJobData(JobID jobID) throws Exception {} + @Override + default void globalCleanup(JobID jobId) throws Exception {} + + @Override + default void localCleanup(JobID jobId) throws Exception {} } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultEntry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultEntry.java new file mode 100644 index 0000000000000..d4cda9a9033ac --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultEntry.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.highavailability; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.util.Preconditions; + +/** + * {@code JobResultEntry} is the entity managed by the {@link JobResultStore}. It collects + * information about a globally terminated job (e.g. {@link JobResult}). + */ +public class JobResultEntry { + + private final JobResult jobResult; + + public JobResultEntry(JobResult jobResult) { + this.jobResult = Preconditions.checkNotNull(jobResult); + } + + public JobResult getJobResult() { + return jobResult; + } + + public JobID getJobId() { + return jobResult.getJobId(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java new file mode 100644 index 0000000000000..077826760f5f8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStore.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.highavailability; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobmaster.JobResult; + +import java.io.IOException; +import java.util.NoSuchElementException; +import java.util.Set; + +/** + * A storage for the results of globally terminated jobs. These results can have the following + * states: + * + *
      + *
    • {@code dirty} - indicating that the corresponding job is not properly cleaned up, yet. + *
    • {@code clean} - indicating that the cleanup of the corresponding job is performed and no + * further actions need to be applied. + *
    + */ +@Internal +public interface JobResultStore { + + /** + * Registers the passed {@link JobResultEntry} instance as {@code dirty} which indicates that + * clean-up operations still need to be performed. Once the job resource cleanup has been + * finalized, we can mark the {@code JobResultEntry} as {@code clean} result using {@link + * #markResultAsClean(JobID)}. + * + * @param jobResultEntry The job result we wish to persist. + * @throws IOException if the creation of the dirty result failed for IO reasons. + * @throws IllegalStateException if the passed {@code jobResultEntry} has a {@code JobID} + * attached that is already registered in this {@code JobResultStore}. + */ + void createDirtyResult(JobResultEntry jobResultEntry) throws IOException, IllegalStateException; + + /** + * Marks an existing {@link JobResultEntry} as {@code clean}. This indicates that no more + * resource cleanup steps need to be performed. No actions should be triggered if the passed + * {@code JobID} belongs to a job that was already marked as clean. + * + * @param jobId Ident of the job we wish to mark as clean. + * @throws IOException if marking the {@code dirty} {@code JobResultEntry} as {@code clean} + * failed for IO reasons. + * @throws NoSuchElementException if there is no corresponding {@code dirty} job present in the + * store for the given {@code JobID}. + */ + void markResultAsClean(JobID jobId) throws IOException, NoSuchElementException; + + /** + * Returns whether the store already contains an entry for a job. + * + * @param jobId Ident of the job we wish to check the store for. + * @return {@code true} if a {@code dirty} or {@code clean} {@link JobResultEntry} exists for + * the given {@code JobID}; otherwise {@code false}. + * @throws IOException if determining whether a job entry is present in the store failed for IO + * reasons. + */ + default boolean hasJobResultEntry(JobID jobId) throws IOException { + return hasDirtyJobResultEntry(jobId) || hasCleanJobResultEntry(jobId); + } + + /** + * Returns whether the store already contains a {@code dirty} entry for the given {@code JobID}. + * + * @param jobId Ident of the job we wish to check the store for. + * @return {@code true}, if a {@code dirty} entry exists for the given {@code JobID}; otherwise + * {@code false}. + * @throws IOException if determining whether a job entry is present in the store failed for IO + * reasons. + */ + boolean hasDirtyJobResultEntry(JobID jobId) throws IOException; + + /** + * Returns whether the store already contains a {@code clean} entry for the given {@code JobID}. + * + * @param jobId Ident of the job we wish to check the store for. + * @return {@code true}, if a {@code clean} entry exists for the given {@code JobID}; otherwise + * {@code false}. + * @throws IOException if determining whether a job entry is present in the store failed for IO + * reasons. + */ + boolean hasCleanJobResultEntry(JobID jobId) throws IOException; + + /** + * Get the persisted {@link JobResult} instances that are marked as {@code dirty}. This is + * useful for recovery of finalization steps. + * + * @return A set of dirty {@code JobResults} from the store. + * @throws IOException if collecting the set of dirty results failed for IO reasons. + */ + Set getDirtyResults() throws IOException; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java deleted file mode 100644 index a2c1aa326fc8a..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/RunningJobsRegistry.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.highavailability; - -import org.apache.flink.api.common.JobID; - -import java.io.IOException; - -/** - * A simple registry that tracks if a certain job is pending execution, running, or completed. - * - *

    This registry is used in highly-available setups with multiple master nodes, to determine - * whether a new leader should attempt to recover a certain job (because the job is still running), - * or whether the job has already finished successfully (in case of a finite job) and the leader has - * only been granted leadership because the previous leader quit cleanly after the job was finished. - * - *

    In addition, the registry can help to determine whether a newly assigned leader JobManager - * should attempt reconciliation with running TaskManagers, or immediately schedule the job from the - * latest checkpoint/savepoint. - */ -public interface RunningJobsRegistry { - - /** The scheduling status of a job, as maintained by the {@code RunningJobsRegistry}. */ - enum JobSchedulingStatus { - - /** Job has not been scheduled, yet. */ - PENDING, - - /** Job has been scheduled and is not yet finished. */ - RUNNING, - - /** Job has been finished, successfully or unsuccessfully. */ - DONE - } - - // ------------------------------------------------------------------------ - - /** - * Marks a job as running. Requesting the job's status via the {@link - * #getJobSchedulingStatus(JobID)} method will return {@link JobSchedulingStatus#RUNNING}. - * - * @param jobID The id of the job. - * @throws IOException Thrown when the communication with the highly-available storage or - * registry failed and could not be retried. - */ - void setJobRunning(JobID jobID) throws IOException; - - /** - * Marks a job as completed. Requesting the job's status via the {@link - * #getJobSchedulingStatus(JobID)} method will return {@link JobSchedulingStatus#DONE}. - * - * @param jobID The id of the job. - * @throws IOException Thrown when the communication with the highly-available storage or - * registry failed and could not be retried. - */ - void setJobFinished(JobID jobID) throws IOException; - - /** - * Gets the scheduling status of a job. - * - * @param jobID The id of the job to check. - * @return The job scheduling status. - * @throws IOException Thrown when the communication with the highly-available storage or - * registry failed and could not be retried. - */ - JobSchedulingStatus getJobSchedulingStatus(JobID jobID) throws IOException; - - /** - * Clear job state form the registry, usually called after job finish. - * - * @param jobID The id of the job to check. - * @throws IOException Thrown when the communication with the highly-available storage or - * registry failed and could not be retried. - */ - void clearJob(JobID jobID) throws IOException; -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java index 188bdbbfd1c42..ddb62b368e5b3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/AbstractNonHaServices.java @@ -23,8 +23,8 @@ import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; -import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry; +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore; @@ -43,14 +43,14 @@ public abstract class AbstractNonHaServices implements HighAvailabilityServices { protected final Object lock = new Object(); - private final RunningJobsRegistry runningJobsRegistry; + private final JobResultStore jobResultStore; private final VoidBlobStore voidBlobStore; private boolean shutdown; public AbstractNonHaServices() { - this.runningJobsRegistry = new StandaloneRunningJobsRegistry(); + this.jobResultStore = new EmbeddedJobResultStore(); this.voidBlobStore = new VoidBlobStore(); shutdown = false; @@ -79,11 +79,11 @@ public JobGraphStore getJobGraphStore() throws Exception { } @Override - public RunningJobsRegistry getRunningJobsRegistry() throws Exception { + public JobResultStore getJobResultStore() throws Exception { synchronized (lock) { checkNotShutdown(); - return runningJobsRegistry; + return jobResultStore; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java new file mode 100644 index 0000000000000..1f54fd62c4cc3 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStore.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.highavailability.nonha.embedded; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.GuardedBy; + +import java.util.HashMap; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +/** A thread-safe in-memory implementation of the {@link JobResultStore}. */ +public class EmbeddedJobResultStore implements JobResultStore { + + private static final Logger LOG = LoggerFactory.getLogger(EmbeddedJobResultStore.class); + + private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); + + @GuardedBy("readWriteLock") + @VisibleForTesting + final Map dirtyJobResults = new HashMap<>(); + + @GuardedBy("readWriteLock") + @VisibleForTesting + final Map cleanJobResults = new HashMap<>(); + + @Override + public void createDirtyResult(JobResultEntry jobResultEntry) { + Preconditions.checkState( + !hasJobResultEntry(jobResultEntry.getJobId()), + "There is already a job registered under the passed ID {}.", + jobResultEntry.getJobId()); + + withWriteLock(() -> dirtyJobResults.put(jobResultEntry.getJobId(), jobResultEntry)); + } + + @Override + public void markResultAsClean(JobID jobId) throws NoSuchElementException { + if (hasCleanJobResultEntry(jobId)) { + LOG.debug("The job {} is already marked as clean. No action required.", jobId); + + return; + } + + withWriteLock( + () -> { + final JobResultEntry jobResultEntry = dirtyJobResults.remove(jobId); + if (jobResultEntry != null) { + cleanJobResults.put(jobId, jobResultEntry); + } else { + throw new NoSuchElementException( + String.format( + "Could not mark job %s as clean as it is not present in the job result store.", + jobId)); + } + }); + } + + @Override + public boolean hasJobResultEntry(JobID jobId) { + return withReadLock( + () -> dirtyJobResults.containsKey(jobId) || cleanJobResults.containsKey(jobId)); + } + + @Override + public boolean hasDirtyJobResultEntry(JobID jobId) { + return withReadLock(() -> dirtyJobResults.containsKey(jobId)); + } + + @Override + public boolean hasCleanJobResultEntry(JobID jobId) { + return withReadLock(() -> cleanJobResults.containsKey(jobId)); + } + + @Override + public Set getDirtyResults() { + return withReadLock( + () -> + dirtyJobResults.values().stream() + .map(JobResultEntry::getJobResult) + .collect(Collectors.toSet())); + } + + private void withWriteLock(Runnable callback) { + readWriteLock.writeLock().lock(); + try { + callback.run(); + } finally { + readWriteLock.writeLock().unlock(); + } + } + + private T withReadLock(Supplier callback) { + readWriteLock.readLock().lock(); + try { + return callback.get(); + } finally { + readWriteLock.readLock().unlock(); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneRunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneRunningJobsRegistry.java deleted file mode 100644 index f5055bab02a71..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/standalone/StandaloneRunningJobsRegistry.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.highavailability.nonha.standalone; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; - -import java.util.HashMap; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** A registry for running jobs, not-highly available. */ -public class StandaloneRunningJobsRegistry implements RunningJobsRegistry { - - /** The currently running jobs. */ - private final HashMap jobStatus = new HashMap<>(); - - @Override - public void setJobRunning(JobID jobID) { - checkNotNull(jobID); - - synchronized (jobStatus) { - jobStatus.put(jobID, JobSchedulingStatus.RUNNING); - } - } - - @Override - public void setJobFinished(JobID jobID) { - checkNotNull(jobID); - - synchronized (jobStatus) { - jobStatus.put(jobID, JobSchedulingStatus.DONE); - } - } - - @Override - public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) { - checkNotNull(jobID); - - synchronized (jobStatus) { - JobSchedulingStatus status = jobStatus.get(jobID); - return status == null ? JobSchedulingStatus.PENDING : status; - } - } - - @Override - public void clearJob(JobID jobID) { - checkNotNull(jobID); - - synchronized (jobStatus) { - jobStatus.remove(jobID); - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java index 13f729e5369d7..b127d848a3451 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/AbstractZooKeeperHaServices.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointRecoveryFactory; import org.apache.flink.runtime.highavailability.AbstractHaServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.util.ZooKeeperUtils; @@ -46,7 +46,7 @@ public AbstractZooKeeperHaServices( Executor executor, Configuration configuration, BlobStoreService blobStoreService) { - super(configuration, executor, blobStoreService); + super(configuration, executor, blobStoreService, new EmbeddedJobResultStore()); this.curatorFrameworkWrapper = checkNotNull(curatorFrameworkWrapper); } @@ -69,12 +69,6 @@ public JobGraphStore createJobGraphStore() throws Exception { curatorFrameworkWrapper.asCuratorFramework(), configuration); } - @Override - public RunningJobsRegistry createRunningJobsRegistry() { - return new ZooKeeperRunningJobsRegistry( - curatorFrameworkWrapper.asCuratorFramework(), configuration); - } - @Override protected void internalClose() throws Exception { curatorFrameworkWrapper.close(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRunningJobsRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRunningJobsRegistry.java deleted file mode 100644 index cf8c3363e7910..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRunningJobsRegistry.java +++ /dev/null @@ -1,153 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.highavailability.zookeeper; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.HighAvailabilityOptions; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; - -import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework; -import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException; -import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.nio.charset.Charset; -import java.util.Arrays; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** A zookeeper based registry for running jobs, highly available. */ -public class ZooKeeperRunningJobsRegistry implements RunningJobsRegistry { - - private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperRunningJobsRegistry.class); - - private static final Charset ENCODING = Charset.forName("utf-8"); - - /** The ZooKeeper client to use. */ - private final CuratorFramework client; - - private final String runningJobPath; - - public ZooKeeperRunningJobsRegistry( - final CuratorFramework client, final Configuration configuration) { - this.client = checkNotNull(client, "client"); - this.runningJobPath = - configuration.getString( - HighAvailabilityOptions.ZOOKEEPER_RUNNING_JOB_REGISTRY_PATH); - } - - @Override - public void setJobRunning(JobID jobID) throws IOException { - checkNotNull(jobID); - - try { - writeEnumToZooKeeper(jobID, JobSchedulingStatus.RUNNING); - } catch (Exception e) { - throw new IOException("Failed to set RUNNING state in ZooKeeper for job " + jobID, e); - } - } - - @Override - public void setJobFinished(JobID jobID) throws IOException { - checkNotNull(jobID); - - try { - writeEnumToZooKeeper(jobID, JobSchedulingStatus.DONE); - } catch (Exception e) { - throw new IOException("Failed to set DONE state in ZooKeeper for job " + jobID, e); - } - } - - @Override - public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) throws IOException { - checkNotNull(jobID); - - try { - final String zkPath = createZkPath(jobID); - final Stat stat = client.checkExists().forPath(zkPath); - if (stat != null) { - // found some data, try to parse it - final byte[] data = client.getData().forPath(zkPath); - if (data != null) { - try { - final String name = new String(data, ENCODING); - return JobSchedulingStatus.valueOf(name); - } catch (IllegalArgumentException e) { - throw new IOException( - "Found corrupt data in ZooKeeper: " - + Arrays.toString(data) - + " is no valid job status"); - } - } - } - - // nothing found, yet, must be in status 'PENDING' - return JobSchedulingStatus.PENDING; - } catch (Exception e) { - throw new IOException("Get finished state from zk fail for job " + jobID.toString(), e); - } - } - - @Override - public void clearJob(JobID jobID) throws IOException { - checkNotNull(jobID); - - final String zkPath = createZkPath(jobID); - - try { - final Stat stat = this.client.checkExists().forPath(zkPath); - if (stat != null) { - this.client.delete().forPath(zkPath); - } - } catch (KeeperException.NoNodeException ignored) { - // The zNode has been deleted by somebody else. This works as well. - } catch (Exception e) { - throw new IOException("Failed to clear job state from ZooKeeper for job " + jobID, e); - } - } - - private String createZkPath(JobID jobID) { - return runningJobPath + jobID.toString(); - } - - private void writeEnumToZooKeeper(JobID jobID, JobSchedulingStatus status) throws Exception { - LOG.debug("Setting scheduling state for job {} to {}.", jobID, status); - final String zkPath = createZkPath(jobID); - while (true) { - try { - final Stat stat = this.client.checkExists().forPath(zkPath); - if (stat != null) { - this.client.setData().forPath(zkPath, status.name().getBytes(ENCODING)); - } else { - this.client - .create() - .creatingParentContainersIfNeeded() - .forPath(zkPath, status.name().getBytes(ENCODING)); - } - return; - } catch (KeeperException.NoNodeException | KeeperException.NodeExistsException e) { - LOG.debug("Retrying failure to set job state from ZooKeeper for job {}", jobID, e); - } - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java index 36fde496789f4..b94191b209b08 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartition.java @@ -43,13 +43,11 @@ import java.nio.ByteBuffer; import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Queue; +import java.util.Random; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; -import java.util.stream.Collectors; -import java.util.stream.IntStream; import static org.apache.flink.runtime.io.network.buffer.Buffer.DataType; import static org.apache.flink.util.Preconditions.checkElementIndex; @@ -68,9 +66,9 @@ public class SortMergeResultPartition extends ResultPartition { /** * Number of expected buffer size to allocate for data writing. Currently, it is an empirical - * value (16M) which can not be configured. + * value (8M) which can not be configured. */ - private static final int NUM_WRITE_BUFFER_BYTES = 16 * 1024 * 1024; + private static final int NUM_WRITE_BUFFER_BYTES = 8 * 1024 * 1024; private final Object lock = new Object(); @@ -141,7 +139,8 @@ public SortMergeResultPartition( // input balance of the downstream tasks this.subpartitionOrder = getRandomSubpartitionOrder(numSubpartitions); this.readScheduler = - new SortMergeResultPartitionReadScheduler(readBufferPool, readIOExecutor, lock); + new SortMergeResultPartitionReadScheduler( + numSubpartitions, readBufferPool, readIOExecutor, lock); PartitionedFileWriter fileWriter = null; try { @@ -492,10 +491,13 @@ public int getNumberOfQueuedBuffers(int targetSubpartition) { } private int[] getRandomSubpartitionOrder(int numSubpartitions) { - List list = - IntStream.range(0, numSubpartitions).boxed().collect(Collectors.toList()); - Collections.shuffle(list); - return list.stream().mapToInt(Integer::intValue).toArray(); + int[] order = new int[numSubpartitions]; + Random random = new Random(); + int shift = random.nextInt(numSubpartitions); + for (int channel = 0; channel < numSubpartitions; ++channel) { + order[(channel + shift) % numSubpartitions] = channel; + } + return order; } @VisibleForTesting diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadScheduler.java index 539a43bf7cef7..8dd33af8a3c20 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadScheduler.java @@ -125,11 +125,15 @@ class SortMergeResultPartitionReadScheduler implements Runnable, BufferRecycler private volatile boolean isReleased; SortMergeResultPartitionReadScheduler( - BatchShuffleReadBufferPool bufferPool, Executor ioExecutor, Object lock) { - this(bufferPool, ioExecutor, lock, DEFAULT_BUFFER_REQUEST_TIMEOUT); + int numSubpartitions, + BatchShuffleReadBufferPool bufferPool, + Executor ioExecutor, + Object lock) { + this(numSubpartitions, bufferPool, ioExecutor, lock, DEFAULT_BUFFER_REQUEST_TIMEOUT); } SortMergeResultPartitionReadScheduler( + int numSubpartitions, BatchShuffleReadBufferPool bufferPool, Executor ioExecutor, Object lock, @@ -138,9 +142,11 @@ class SortMergeResultPartitionReadScheduler implements Runnable, BufferRecycler this.lock = checkNotNull(lock); this.bufferPool = checkNotNull(bufferPool); this.ioExecutor = checkNotNull(ioExecutor); - // one partition reader can consume at most 32M (the expected buffers per request is 8M) - // buffers for data read. Currently, it is only an empirical value can not be configured - this.maxRequestedBuffers = Math.max(1, 4 * bufferPool.getNumBuffersPerRequest()); + // one partition reader can consume at most Math.max(16M, numSubpartitions) (the expected + // buffers per request is 8M) buffers for data read, which means larger parallelism, more + // buffers. Currently, it is only an empirical strategy which can not be configured. + this.maxRequestedBuffers = + Math.max(2 * bufferPool.getNumBuffersPerRequest(), numSubpartitions); this.bufferRequestTimeout = checkNotNull(bufferRequestTimeout); // initialize the buffer pool eagerly to avoid reporting errors like OOM too late diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java index 7b70d35886377..5991ca81f7e97 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java @@ -43,7 +43,7 @@ *

    For each channel, the consumption life cycle is as follows: * *

      - *
    1. {@link #requestSubpartition(int)} + *
    2. {@link #requestSubpartition()} *
    3. {@link #getNextBuffer()} *
    4. {@link #releaseAllResources()} *
    @@ -52,8 +52,12 @@ public abstract class InputChannel { /** The info of the input channel to identify it globally within a task. */ protected final InputChannelInfo channelInfo; + /** The parent partition of the subpartition consumed by this channel. */ protected final ResultPartitionID partitionId; + /** The index of the subpartition consumed by this channel. */ + protected final int consumedSubpartitionIndex; + protected final SingleInputGate inputGate; // - Asynchronous error notification -------------------------------------- @@ -79,6 +83,7 @@ protected InputChannel( SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, int initialBackoff, int maxBackoff, Counter numBytesIn, @@ -95,6 +100,9 @@ protected InputChannel( this.channelInfo = new InputChannelInfo(inputGate.getGateIndex(), channelIndex); this.partitionId = checkNotNull(partitionId); + checkArgument(consumedSubpartitionIndex >= 0); + this.consumedSubpartitionIndex = consumedSubpartitionIndex; + this.initialBackoff = initial; this.maxBackoff = max; this.currentBackoff = initial == 0 ? -1 : 0; @@ -124,6 +132,10 @@ public ResultPartitionID getPartitionId() { return partitionId; } + public int getConsumedSubpartitionIndex() { + return consumedSubpartitionIndex; + } + /** * After sending a {@link org.apache.flink.runtime.io.network.api.CheckpointBarrier} of * exactly-once mode, the upstream will be blocked and become unavailable. This method tries to @@ -163,13 +175,10 @@ protected void notifyBufferAvailable(int numAvailableBuffers) throws IOException // ------------------------------------------------------------------------ /** - * Requests the queue with the specified index of the source intermediate result partition. - * - *

    The queue index to request depends on which sub task the channel belongs to and is - * specified by the consumer of this channel. + * Requests the subpartition specified by {@link #partitionId} and {@link + * #consumedSubpartitionIndex}. */ - abstract void requestSubpartition(int subpartitionIndex) - throws IOException, InterruptedException; + abstract void requestSubpartition() throws IOException, InterruptedException; /** * Returns the next buffer from the consumed subpartition or {@code Optional.empty()} if there diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java index 13c6538a8fc06..f409a68524f97 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java @@ -76,6 +76,7 @@ public LocalInputChannel( SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, ResultPartitionManager partitionManager, TaskEventPublisher taskEventPublisher, int initialBackoff, @@ -88,6 +89,7 @@ public LocalInputChannel( inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, initialBackoff, maxBackoff, numBytesIn, @@ -111,7 +113,7 @@ public void checkpointStopped(long checkpointId) { } @Override - protected void requestSubpartition(int subpartitionIndex) throws IOException { + protected void requestSubpartition() throws IOException { boolean retriggerRequest = false; boolean notifyDataAvailable = false; @@ -124,14 +126,14 @@ protected void requestSubpartition(int subpartitionIndex) throws IOException { LOG.debug( "{}: Requesting LOCAL subpartition {} of partition {}. {}", this, - subpartitionIndex, + consumedSubpartitionIndex, partitionId, channelStatePersister); try { ResultSubpartitionView subpartitionView = partitionManager.createSubpartitionView( - partitionId, subpartitionIndex, this); + partitionId, consumedSubpartitionIndex, this); if (subpartitionView == null) { throw new IOException("Error requesting subpartition."); @@ -165,12 +167,13 @@ protected void requestSubpartition(int subpartitionIndex) throws IOException { // deadlock with a concurrent release of the channel via the // input gate. if (retriggerRequest) { - inputGate.retriggerPartitionRequest(partitionId.getPartitionId()); + inputGate.retriggerPartitionRequest( + partitionId.getPartitionId(), consumedSubpartitionIndex); } } /** Retriggers a subpartition request. */ - void retriggerSubpartitionRequest(Timer timer, final int subpartitionIndex) { + void retriggerSubpartitionRequest(Timer timer) { synchronized (requestLock) { checkState(subpartitionView == null, "already requested partition"); @@ -179,7 +182,7 @@ void retriggerSubpartitionRequest(Timer timer, final int subpartitionIndex) { @Override public void run() { try { - requestSubpartition(subpartitionIndex); + requestSubpartition(); } catch (Throwable t) { setError(t); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalRecoveredInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalRecoveredInputChannel.java index 1b790a9e91e5c..fa106188b03eb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalRecoveredInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalRecoveredInputChannel.java @@ -37,6 +37,7 @@ public class LocalRecoveredInputChannel extends RecoveredInputChannel { SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, ResultPartitionManager partitionManager, TaskEventPublisher taskEventPublisher, int initialBackOff, @@ -47,6 +48,7 @@ public class LocalRecoveredInputChannel extends RecoveredInputChannel { inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, initialBackOff, maxBackoff, metrics.getNumBytesInLocalCounter(), @@ -63,6 +65,7 @@ protected InputChannel toInputChannelInternal() { inputGate, getChannelIndex(), partitionId, + consumedSubpartitionIndex, partitionManager, taskEventPublisher, initialBackoff, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannel.java index 7f31116d2d763..469b05e96a19d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannel.java @@ -76,6 +76,7 @@ public abstract class RecoveredInputChannel extends InputChannel implements Chan SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, int initialBackoff, int maxBackoff, Counter numBytesIn, @@ -85,6 +86,7 @@ public abstract class RecoveredInputChannel extends InputChannel implements Chan inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, initialBackoff, maxBackoff, numBytesIn, @@ -223,7 +225,7 @@ public void acknowledgeAllRecordsProcessed() throws IOException { } @Override - final void requestSubpartition(int subpartitionIndex) { + final void requestSubpartition() { throw new UnsupportedOperationException( "RecoveredInputChannel should never request partition."); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java index a9bce9935b7ab..5b476b5db0a70 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java @@ -119,6 +119,7 @@ public RemoteInputChannel( SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, ConnectionID connectionId, ConnectionManager connectionManager, int initialBackOff, @@ -132,6 +133,7 @@ public RemoteInputChannel( inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, initialBackOff, maxBackoff, numBytesIn, @@ -170,13 +172,12 @@ void setup() throws IOException { /** Requests a remote subpartition. */ @VisibleForTesting @Override - public void requestSubpartition(int subpartitionIndex) - throws IOException, InterruptedException { + public void requestSubpartition() throws IOException, InterruptedException { if (partitionRequestClient == null) { LOG.debug( "{}: Requesting REMOTE subpartition {} of partition {}. {}", this, - subpartitionIndex, + consumedSubpartitionIndex, partitionId, channelStatePersister); // Create a client and request the partition @@ -189,17 +190,18 @@ public void requestSubpartition(int subpartitionIndex) throw new PartitionConnectionException(partitionId, e); } - partitionRequestClient.requestSubpartition(partitionId, subpartitionIndex, this, 0); + partitionRequestClient.requestSubpartition( + partitionId, consumedSubpartitionIndex, this, 0); } } /** Retriggers a remote subpartition request. */ - void retriggerSubpartitionRequest(int subpartitionIndex) throws IOException { + void retriggerSubpartitionRequest() throws IOException { checkPartitionRequestQueueInitialized(); if (increaseBackoff()) { partitionRequestClient.requestSubpartition( - partitionId, subpartitionIndex, this, getCurrentBackoff()); + partitionId, consumedSubpartitionIndex, this, getCurrentBackoff()); } else { failPartitionRequest(); } @@ -797,7 +799,7 @@ public void onEmptyBuffer(int sequenceNumber, int backlog) throws IOException { } public void onFailedPartitionRequest() { - inputGate.triggerPartitionStateCheck(partitionId); + inputGate.triggerPartitionStateCheck(partitionId, consumedSubpartitionIndex); } public void onError(Throwable cause) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java index 18ac6bf42e376..2c0efa01592ab 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteRecoveredInputChannel.java @@ -39,6 +39,7 @@ public class RemoteRecoveredInputChannel extends RecoveredInputChannel { SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, ConnectionID connectionId, ConnectionManager connectionManager, int initialBackOff, @@ -49,6 +50,7 @@ public class RemoteRecoveredInputChannel extends RecoveredInputChannel { inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, initialBackOff, maxBackoff, metrics.getNumBytesInRemoteCounter(), @@ -66,6 +68,7 @@ protected InputChannel toInputChannelInternal() throws IOException { inputGate, getChannelIndex(), partitionId, + consumedSubpartitionIndex, connectionId, connectionManager, initialBackoff, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java index 00e05ea745324..09f8144337ae8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGate.java @@ -24,6 +24,7 @@ import org.apache.flink.core.memory.MemorySegmentProvider; import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.deployment.SubpartitionIndexRange; import org.apache.flink.runtime.event.AbstractEvent; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.execution.CancelTaskException; @@ -134,19 +135,20 @@ public class SingleInputGate extends IndexedInputGate { private final ResultPartitionType consumedPartitionType; /** - * The index of the consumed subpartition of each consumed partition. This index depends on the - * {@link DistributionPattern} and the subtask indices of the producing and consuming task. + * Range of the index of the consumed subpartition of each consumed partition. This index + * depends on the {@link DistributionPattern} and the subtask indices of the producing and + * consuming task. The range is inclusive. */ - private final int consumedSubpartitionIndex; + private final SubpartitionIndexRange subpartitionIndexRange; /** The number of input channels (equivalent to the number of consumed partitions). */ private final int numberOfInputChannels; /** - * Input channels. There is a one input channel for each consumed intermediate result partition. - * We store this in a map for runtime updates of single channels. + * Input channels. There is one input channel for each consumed subpartition. We store this in a + * map for runtime updates of single channels. */ - private final Map inputChannels; + private final Map inputChannels; @GuardedBy("requestLock") private final InputChannel[] channels; @@ -216,7 +218,7 @@ public SingleInputGate( int gateIndex, IntermediateDataSetID consumedResultId, final ResultPartitionType consumedPartitionType, - int consumedSubpartitionIndex, + SubpartitionIndexRange subpartitionIndexRange, int numberOfInputChannels, PartitionProducerStateProvider partitionProducerStateProvider, SupplierWithException bufferPoolFactory, @@ -234,8 +236,7 @@ public SingleInputGate( this.consumedPartitionType = checkNotNull(consumedPartitionType); this.bufferPoolFactory = checkNotNull(bufferPoolFactory); - checkArgument(consumedSubpartitionIndex >= 0); - this.consumedSubpartitionIndex = consumedSubpartitionIndex; + this.subpartitionIndexRange = checkNotNull(subpartitionIndexRange); checkArgument(numberOfInputChannels > 0); this.numberOfInputChannels = numberOfInputChannels; @@ -318,8 +319,7 @@ public void requestPartitions() { @VisibleForTesting public void convertRecoveredInputChannels() { LOG.debug("Converting recovered input channels ({} channels)", getNumberOfInputChannels()); - for (Map.Entry entry : - inputChannels.entrySet()) { + for (Map.Entry entry : inputChannels.entrySet()) { InputChannel inputChannel = entry.getValue(); if (inputChannel instanceof RecoveredInputChannel) { try { @@ -339,7 +339,7 @@ public void convertRecoveredInputChannels() { private void internalRequestPartitions() { for (InputChannel inputChannel : inputChannels.values()) { try { - inputChannel.requestSubpartition(consumedSubpartitionIndex); + inputChannel.requestSubpartition(); } catch (Throwable t) { inputChannel.setError(t); return; @@ -539,7 +539,11 @@ public void setInputChannels(InputChannel... channels) { for (InputChannel inputChannel : channels) { IntermediateResultPartitionID partitionId = inputChannel.getPartitionId().getPartitionId(); - if (inputChannels.put(partitionId, inputChannel) == null + int subpartitionIndex = inputChannel.getConsumedSubpartitionIndex(); + if (inputChannels.put( + new SubpartitionInfo(partitionId, subpartitionIndex), + inputChannel) + == null && inputChannel instanceof UnknownInputChannel) { numberOfUninitializedChannels++; @@ -560,47 +564,55 @@ public void updateInputChannel( IntermediateResultPartitionID partitionId = shuffleDescriptor.getResultPartitionID().getPartitionId(); - InputChannel current = inputChannels.get(partitionId); - - if (current instanceof UnknownInputChannel) { - UnknownInputChannel unknownChannel = (UnknownInputChannel) current; - boolean isLocal = shuffleDescriptor.isLocalTo(localLocation); - InputChannel newChannel; - if (isLocal) { - newChannel = unknownChannel.toLocalInputChannel(); - } else { - RemoteInputChannel remoteInputChannel = - unknownChannel.toRemoteInputChannel( - shuffleDescriptor.getConnectionId()); - remoteInputChannel.setup(); - newChannel = remoteInputChannel; - } - LOG.debug("{}: Updated unknown input channel to {}.", owningTaskName, newChannel); + for (int subpartitionIndex = subpartitionIndexRange.getStartIndex(); + subpartitionIndex <= subpartitionIndexRange.getEndIndex(); + ++subpartitionIndex) { + SubpartitionInfo subpartitionInfo = + new SubpartitionInfo(partitionId, subpartitionIndex); + InputChannel current = inputChannels.get(subpartitionInfo); + + if (current instanceof UnknownInputChannel) { + UnknownInputChannel unknownChannel = (UnknownInputChannel) current; + boolean isLocal = shuffleDescriptor.isLocalTo(localLocation); + InputChannel newChannel; + if (isLocal) { + newChannel = unknownChannel.toLocalInputChannel(); + } else { + RemoteInputChannel remoteInputChannel = + unknownChannel.toRemoteInputChannel( + shuffleDescriptor.getConnectionId()); + remoteInputChannel.setup(); + newChannel = remoteInputChannel; + } + LOG.debug( + "{}: Updated unknown input channel to {}.", owningTaskName, newChannel); - inputChannels.put(partitionId, newChannel); - channels[current.getChannelIndex()] = newChannel; + inputChannels.put(subpartitionInfo, newChannel); + channels[current.getChannelIndex()] = newChannel; - if (requestedPartitionsFlag) { - newChannel.requestSubpartition(consumedSubpartitionIndex); - } + if (requestedPartitionsFlag) { + newChannel.requestSubpartition(); + } - for (TaskEvent event : pendingEvents) { - newChannel.sendTaskEvent(event); - } + for (TaskEvent event : pendingEvents) { + newChannel.sendTaskEvent(event); + } - if (--numberOfUninitializedChannels == 0) { - pendingEvents.clear(); + if (--numberOfUninitializedChannels == 0) { + pendingEvents.clear(); + } } } } } /** Retriggers a partition request. */ - public void retriggerPartitionRequest(IntermediateResultPartitionID partitionId) - throws IOException { + public void retriggerPartitionRequest( + IntermediateResultPartitionID partitionId, int subpartitionIndex) throws IOException { synchronized (requestLock) { if (!closeFuture.isDone()) { - final InputChannel ch = inputChannels.get(partitionId); + final InputChannel ch = + inputChannels.get(new SubpartitionInfo(partitionId, subpartitionIndex)); checkNotNull(ch, "Unknown input channel with ID " + partitionId); @@ -608,11 +620,11 @@ public void retriggerPartitionRequest(IntermediateResultPartitionID partitionId) "{}: Retriggering partition request {}:{}.", owningTaskName, ch.partitionId, - consumedSubpartitionIndex); + ch.getConsumedSubpartitionIndex()); if (ch.getClass() == RemoteInputChannel.class) { final RemoteInputChannel rch = (RemoteInputChannel) ch; - rch.retriggerSubpartitionRequest(consumedSubpartitionIndex); + rch.retriggerSubpartitionRequest(); } else if (ch.getClass() == LocalInputChannel.class) { final LocalInputChannel ich = (LocalInputChannel) ch; @@ -620,8 +632,7 @@ public void retriggerPartitionRequest(IntermediateResultPartitionID partitionId) retriggerLocalRequestTimer = new Timer(true); } - ich.retriggerSubpartitionRequest( - retriggerLocalRequestTimer, consumedSubpartitionIndex); + ich.retriggerSubpartitionRequest(retriggerLocalRequestTimer); } else { throw new IllegalStateException( "Unexpected type of channel to retrigger partition: " + ch.getClass()); @@ -962,7 +973,7 @@ void notifyPriorityEventForce(InputChannel inputChannel) { queueChannel(checkNotNull(inputChannel), null, true); } - void triggerPartitionStateCheck(ResultPartitionID partitionId) { + void triggerPartitionStateCheck(ResultPartitionID partitionId, int subpartitionIndex) { partitionProducerStateProvider.requestPartitionProducerState( consumedResultId, partitionId, @@ -972,7 +983,8 @@ void triggerPartitionStateCheck(ResultPartitionID partitionId) { .isProducerReadyOrAbortConsumption(responseHandle); if (isProducingState) { try { - retriggerPartitionRequest(partitionId.getPartitionId()); + retriggerPartitionRequest( + partitionId.getPartitionId(), subpartitionIndex); } catch (IOException t) { responseHandle.failConsumption(t); } @@ -1073,7 +1085,36 @@ private Optional getChannel(boolean blocking) throws InterruptedEx // ------------------------------------------------------------------------ - public Map getInputChannels() { + public Map getInputChannels() { return inputChannels; } + + static class SubpartitionInfo { + private final IntermediateResultPartitionID partitionID; + private final int subpartitionIndex; + + SubpartitionInfo(IntermediateResultPartitionID partitionID, int subpartitionIndex) { + this.partitionID = checkNotNull(partitionID); + checkArgument(subpartitionIndex >= 0); + this.subpartitionIndex = subpartitionIndex; + } + + @Override + public int hashCode() { + return partitionID.hashCode() ^ subpartitionIndex; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } else if (obj != null && obj.getClass() == getClass()) { + SubpartitionInfo that = (SubpartitionInfo) obj; + return that.partitionID.equals(this.partitionID) + && that.subpartitionIndex == this.subpartitionIndex; + } else { + return false; + } + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java index 17a9c1a8fd7b9..d862af37be066 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java @@ -22,6 +22,7 @@ import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; +import org.apache.flink.runtime.deployment.SubpartitionIndexRange; import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.TaskEventPublisher; @@ -42,6 +43,7 @@ import org.apache.flink.runtime.throughput.BufferDebloatConfiguration; import org.apache.flink.runtime.throughput.BufferDebloater; import org.apache.flink.runtime.throughput.ThroughputCalculator; +import org.apache.flink.util.MathUtils; import org.apache.flink.util.clock.SystemClock; import org.apache.flink.util.function.SupplierWithException; @@ -126,14 +128,17 @@ public SingleInputGate create( final String owningTaskName = owner.getOwnerName(); final MetricGroup networkInputGroup = owner.getInputGroup(); + + SubpartitionIndexRange subpartitionIndexRange = igdd.getConsumedSubpartitionIndexRange(); SingleInputGate inputGate = new SingleInputGate( owningTaskName, gateIndex, igdd.getConsumedResultId(), igdd.getConsumedPartitionType(), - igdd.getConsumedSubpartitionIndex(), - igdd.getShuffleDescriptors().length, + subpartitionIndexRange, + calculateNumChannels( + igdd.getShuffleDescriptors().length, subpartitionIndexRange), partitionProducerStateProvider, bufferPoolFactory, bufferDecompressor, @@ -145,7 +150,7 @@ public SingleInputGate create( InputChannelMetrics metrics = new InputChannelMetrics(networkInputGroup, owner.getParentGroup()); - createInputChannels(owningTaskName, igdd, inputGate, metrics); + createInputChannels(owningTaskName, igdd, inputGate, subpartitionIndexRange, metrics); return inputGate; } @@ -173,20 +178,35 @@ private void createInputChannels( String owningTaskName, InputGateDeploymentDescriptor inputGateDeploymentDescriptor, SingleInputGate inputGate, + SubpartitionIndexRange subpartitionIndexRange, InputChannelMetrics metrics) { ShuffleDescriptor[] shuffleDescriptors = inputGateDeploymentDescriptor.getShuffleDescriptors(); - // Create the input channels. There is one input channel for each consumed partition. - InputChannel[] inputChannels = new InputChannel[shuffleDescriptors.length]; + // Create the input channels. There is one input channel for each consumed subpartition. + InputChannel[] inputChannels = + new InputChannel + [calculateNumChannels(shuffleDescriptors.length, subpartitionIndexRange)]; ChannelStatistics channelStatistics = new ChannelStatistics(); - for (int i = 0; i < inputChannels.length; i++) { - inputChannels[i] = - createInputChannel( - inputGate, i, shuffleDescriptors[i], channelStatistics, metrics); + int channelIdx = 0; + for (int i = 0; i < shuffleDescriptors.length; ++i) { + for (int subpartitionIndex = subpartitionIndexRange.getStartIndex(); + subpartitionIndex <= subpartitionIndexRange.getEndIndex(); + ++subpartitionIndex) { + inputChannels[channelIdx] = + createInputChannel( + inputGate, + channelIdx, + shuffleDescriptors[i], + subpartitionIndex, + channelStatistics, + metrics); + channelIdx++; + } } + inputGate.setInputChannels(inputChannels); LOG.debug( @@ -200,6 +220,7 @@ private InputChannel createInputChannel( SingleInputGate inputGate, int index, ShuffleDescriptor shuffleDescriptor, + int consumedSubpartitionIndex, ChannelStatistics channelStatistics, InputChannelMetrics metrics) { return applyWithShuffleTypeCheck( @@ -211,6 +232,7 @@ private InputChannel createInputChannel( inputGate, index, unknownShuffleDescriptor.getResultPartitionID(), + consumedSubpartitionIndex, partitionManager, taskEventPublisher, connectionManager, @@ -224,15 +246,23 @@ private InputChannel createInputChannel( inputGate, index, nettyShuffleDescriptor, + consumedSubpartitionIndex, channelStatistics, metrics)); } + private static int calculateNumChannels( + int numShuffleDescriptors, SubpartitionIndexRange subpartitionIndexRange) { + return MathUtils.checkedDownCast( + ((long) numShuffleDescriptors) * subpartitionIndexRange.size()); + } + @VisibleForTesting protected InputChannel createKnownInputChannel( SingleInputGate inputGate, int index, NettyShuffleDescriptor inputChannelDescriptor, + int consumedSubpartitionIndex, ChannelStatistics channelStatistics, InputChannelMetrics metrics) { ResultPartitionID partitionId = inputChannelDescriptor.getResultPartitionID(); @@ -243,6 +273,7 @@ protected InputChannel createKnownInputChannel( inputGate, index, partitionId, + consumedSubpartitionIndex, partitionManager, taskEventPublisher, partitionRequestInitialBackoff, @@ -256,6 +287,7 @@ protected InputChannel createKnownInputChannel( inputGate, index, partitionId, + consumedSubpartitionIndex, inputChannelDescriptor.getConnectionId(), connectionManager, partitionRequestInitialBackoff, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java index 88d91c7c3916f..18696072cd712 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java @@ -66,6 +66,7 @@ public UnknownInputChannel( SingleInputGate gate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, ResultPartitionManager partitionManager, TaskEventPublisher taskEventPublisher, ConnectionManager connectionManager, @@ -74,7 +75,15 @@ public UnknownInputChannel( int networkBuffersPerChannel, InputChannelMetrics metrics) { - super(gate, channelIndex, partitionId, initialBackoff, maxBackoff, null, null); + super( + gate, + channelIndex, + partitionId, + consumedSubpartitionIndex, + initialBackoff, + maxBackoff, + null, + null); this.partitionManager = checkNotNull(partitionManager); this.taskEventPublisher = checkNotNull(taskEventPublisher); @@ -97,7 +106,7 @@ public void acknowledgeAllRecordsProcessed() throws IOException { } @Override - public void requestSubpartition(int subpartitionIndex) throws IOException { + public void requestSubpartition() throws IOException { // Nothing to do here } @@ -154,6 +163,7 @@ public RemoteInputChannel toRemoteInputChannel(ConnectionID producerAddress) { inputGate, getChannelIndex(), partitionId, + consumedSubpartitionIndex, checkNotNull(producerAddress), connectionManager, initialBackoff, @@ -169,6 +179,7 @@ public LocalInputChannel toLocalInputChannel() { inputGate, getChannelIndex(), partitionId, + consumedSubpartitionIndex, partitionManager, taskEventPublisher, initialBackoff, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java index 832f9c92baf97..5416bdae4261e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java @@ -238,7 +238,7 @@ public void putJobGraph(JobGraph jobGraph) throws Exception { } @Override - public void removeJobGraph(JobID jobId) throws Exception { + public void globalCleanup(JobID jobId) throws Exception { checkNotNull(jobId, "Job ID"); String name = jobGraphStoreUtil.jobIDToName(jobId); @@ -262,7 +262,7 @@ public void removeJobGraph(JobID jobId) throws Exception { } @Override - public void releaseJobGraph(JobID jobId) throws Exception { + public void localCleanup(JobID jobId) throws Exception { checkNotNull(jobId, "Job ID"); LOG.debug("Releasing job graph {} from {}.", jobId, jobGraphStateHandleStore); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobGraphStoreFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobPersistenceComponentFactory.java similarity index 59% rename from flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobGraphStoreFactory.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobPersistenceComponentFactory.java index ae8d6ca5cc7a7..05dce66c3aac4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobGraphStoreFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobPersistenceComponentFactory.java @@ -19,28 +19,40 @@ package org.apache.flink.runtime.jobmanager; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.function.SupplierWithException; /** - * {@link JobGraphStoreFactory} implementation which creates a {@link JobGraphStore} using the - * provided {@link HighAvailabilityServices}. + * {@link JobPersistenceComponentFactory} implementation which creates a {@link JobGraphStore} using + * the provided {@link HighAvailabilityServices}. */ -public class HaServicesJobGraphStoreFactory implements JobGraphStoreFactory { +public class HaServicesJobPersistenceComponentFactory implements JobPersistenceComponentFactory { private final HighAvailabilityServices highAvailabilityServices; - public HaServicesJobGraphStoreFactory(HighAvailabilityServices highAvailabilityServices) { + public HaServicesJobPersistenceComponentFactory( + HighAvailabilityServices highAvailabilityServices) { this.highAvailabilityServices = highAvailabilityServices; } @Override - public JobGraphStore create() { + public JobGraphStore createJobGraphStore() { + return create(highAvailabilityServices::getJobGraphStore, JobGraphStore.class); + } + + @Override + public JobResultStore createJobResultStore() { + return create(highAvailabilityServices::getJobResultStore, JobResultStore.class); + } + + private T create(SupplierWithException supplier, Class clazz) { try { - return highAvailabilityServices.getJobGraphStore(); + return supplier.get(); } catch (Exception e) { throw new FlinkRuntimeException( String.format( "Could not create %s from %s.", - JobGraphStore.class.getSimpleName(), + clazz.getSimpleName(), highAvailabilityServices.getClass().getSimpleName()), e); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphWriter.java index 23542a330b923..63d644a813c07 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphWriter.java @@ -19,10 +19,12 @@ package org.apache.flink.runtime.jobmanager; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.dispatcher.cleanup.GloballyCleanableResource; +import org.apache.flink.runtime.dispatcher.cleanup.LocallyCleanableResource; import org.apache.flink.runtime.jobgraph.JobGraph; /** Allows to store and remove job graphs. */ -public interface JobGraphWriter { +public interface JobGraphWriter extends LocallyCleanableResource, GloballyCleanableResource { /** * Adds the {@link JobGraph} instance. * @@ -30,9 +32,6 @@ public interface JobGraphWriter { */ void putJobGraph(JobGraph jobGraph) throws Exception; - /** Removes the {@link JobGraph} with the given {@link JobID} if it exists. */ - void removeJobGraph(JobID jobId) throws Exception; - /** * Releases the locks on the specified {@link JobGraph}. * @@ -42,5 +41,9 @@ public interface JobGraphWriter { * @param jobId specifying the job to release the locks for * @throws Exception if the locks cannot be released */ - void releaseJobGraph(JobID jobId) throws Exception; + @Override + default void localCleanup(JobID jobId) throws Exception {} + + @Override + default void globalCleanup(JobID jobId) throws Exception {} } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphStoreFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobPersistenceComponentFactory.java similarity index 66% rename from flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphStoreFactory.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobPersistenceComponentFactory.java index 1530509003981..e49c12a1e65a2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobGraphStoreFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobPersistenceComponentFactory.java @@ -18,13 +18,22 @@ package org.apache.flink.runtime.jobmanager; -/** Factory for {@link JobGraphStore}. */ -public interface JobGraphStoreFactory { +import org.apache.flink.runtime.highavailability.JobResultStore; + +/** Factory for components that are responsible for persisting a job for recovery. */ +public interface JobPersistenceComponentFactory { /** * Creates a {@link JobGraphStore}. * - * @return a {@link JobGraphStore} instance + * @return a {@code JobGraphStore} instance + */ + JobGraphStore createJobGraphStore(); + + /** + * Creates {@link JobResultStore} instances. + * + * @return a {@code JobResultStore} instance. */ - JobGraphStore create(); + JobResultStore createJobResultStore(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStore.java index 656df2ffd3607..f7d8135136aa4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStore.java @@ -47,16 +47,6 @@ public void putJobGraph(JobGraph jobGraph) { // Nothing to do } - @Override - public void removeJobGraph(JobID jobId) { - // Nothing to do - } - - @Override - public void releaseJobGraph(JobID jobId) { - // nothing to do - } - @Override public Collection getJobIds() { return Collections.emptyList(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ThrowingJobGraphWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ThrowingJobGraphWriter.java index 224b7fbd224f5..8bf9aa0c45d2f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ThrowingJobGraphWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ThrowingJobGraphWriter.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.jobmanager; -import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobgraph.JobGraph; /** {@link JobGraphWriter} implementation which does not allow to store {@link JobGraph}. */ @@ -29,10 +28,4 @@ public enum ThrowingJobGraphWriter implements JobGraphWriter { public void putJobGraph(JobGraph jobGraph) { throw new UnsupportedOperationException("Cannot store job graphs."); } - - @Override - public void removeJobGraph(JobID jobId) {} - - @Override - public void releaseJobGraph(JobID jobId) {} } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java index 45782d4aff405..39998bed9f785 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunner.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobmaster.factories.JobMasterServiceProcessFactory; import org.apache.flink.runtime.leaderelection.LeaderContender; import org.apache.flink.runtime.leaderelection.LeaderElectionService; @@ -81,7 +81,7 @@ public class JobMasterServiceLeadershipRunner implements JobManagerRunner, Leade private final LeaderElectionService leaderElectionService; - private final RunningJobsRegistry runningJobsRegistry; + private final JobResultStore jobResultStore; private final LibraryCacheManager.ClassLoaderLease classLoaderLease; @@ -111,12 +111,12 @@ public class JobMasterServiceLeadershipRunner implements JobManagerRunner, Leade public JobMasterServiceLeadershipRunner( JobMasterServiceProcessFactory jobMasterServiceProcessFactory, LeaderElectionService leaderElectionService, - RunningJobsRegistry runningJobsRegistry, + JobResultStore jobResultStore, LibraryCacheManager.ClassLoaderLease classLoaderLease, FatalErrorHandler fatalErrorHandler) { this.jobMasterServiceProcessFactory = jobMasterServiceProcessFactory; this.leaderElectionService = leaderElectionService; - this.runningJobsRegistry = runningJobsRegistry; + this.jobResultStore = jobResultStore; this.classLoaderLease = classLoaderLease; this.fatalErrorHandler = fatalErrorHandler; } @@ -269,13 +269,17 @@ private void startJobMasterServiceProcessAsync(UUID leaderSessionId) { @GuardedBy("lock") private void verifyJobSchedulingStatusAndCreateJobMasterServiceProcess(UUID leaderSessionId) throws FlinkException { - final RunningJobsRegistry.JobSchedulingStatus jobSchedulingStatus = - getJobSchedulingStatus(); - - if (jobSchedulingStatus == RunningJobsRegistry.JobSchedulingStatus.DONE) { - jobAlreadyDone(); - } else { - createNewJobMasterServiceProcess(leaderSessionId); + try { + if (jobResultStore.hasJobResultEntry(getJobID())) { + jobAlreadyDone(); + } else { + createNewJobMasterServiceProcess(leaderSessionId); + } + } catch (IOException e) { + throw new FlinkException( + String.format( + "Could not retrieve the job scheduling status for job %s.", getJobID()), + e); } } @@ -293,17 +297,6 @@ private void jobAlreadyDone() { new JobAlreadyDoneException(getJobID()))))); } - private RunningJobsRegistry.JobSchedulingStatus getJobSchedulingStatus() throws FlinkException { - try { - return runningJobsRegistry.getJobSchedulingStatus(getJobID()); - } catch (IOException e) { - throw new FlinkException( - String.format( - "Could not retrieve the job scheduling status for job %s.", getJobID()), - e); - } - } - @GuardedBy("lock") private void createNewJobMasterServiceProcess(UUID leaderSessionId) throws FlinkException { Preconditions.checkState(jobMasterServiceProcess.closeAsync().isDone()); @@ -312,16 +305,6 @@ private void createNewJobMasterServiceProcess(UUID leaderSessionId) throws Flink "Create new JobMasterServiceProcess because we were granted leadership under {}.", leaderSessionId); - try { - runningJobsRegistry.setJobRunning(getJobID()); - } catch (IOException e) { - throw new FlinkException( - String.format( - "Failed to set the job %s to running in the running jobs registry.", - getJobID()), - e); - } - jobMasterServiceProcess = jobMasterServiceProcessFactory.create(leaderSessionId); forwardIfValidLeader( @@ -382,17 +365,7 @@ private void onJobCompletion( "Could not retrieve JobMasterGateway because the JobMaster failed.", throwable)); } else { - if (jobManagerRunnerResult.isSuccess()) { - try { - runningJobsRegistry.setJobFinished(getJobID()); - } catch (IOException e) { - LOG.error( - "Could not un-register from high-availability services job {}." - + "Other JobManager's may attempt to recover it and re-execute it.", - getJobID(), - e); - } - } else { + if (!jobManagerRunnerResult.isSuccess()) { jobMasterGatewayFuture.completeExceptionally( new FlinkException( "Could not retrieve JobMasterGateway because the JobMaster initialization failed.", diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceProcessFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceProcessFactory.java index 1c60ccf698fc3..cbd7920e69dec 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceProcessFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/factories/DefaultJobMasterServiceProcessFactory.java @@ -68,7 +68,7 @@ public JobID getJobId() { @Override public ArchivedExecutionGraph createArchivedExecutionGraph( JobStatus jobStatus, @Nullable Throwable cause) { - return ArchivedExecutionGraph.createFromInitializingJob( + return ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, jobName, jobStatus, cause, checkpointingSettings, initializationTimestamp); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/JobManagerMetricGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/JobManagerMetricGroup.java index 431dab46bcce2..0ac161402ce1d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/JobManagerMetricGroup.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/JobManagerMetricGroup.java @@ -20,6 +20,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.metrics.CharacterFilter; +import org.apache.flink.runtime.dispatcher.cleanup.GloballyCleanableResource; +import org.apache.flink.runtime.dispatcher.cleanup.LocallyCleanableResource; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.dump.QueryScopeInfo; import org.apache.flink.runtime.metrics.scope.ScopeFormat; @@ -33,7 +35,8 @@ *

    Contains extra logic for adding jobs with tasks, and removing jobs when they do not contain * tasks any more */ -public class JobManagerMetricGroup extends ComponentMetricGroup { +public class JobManagerMetricGroup extends ComponentMetricGroup + implements LocallyCleanableResource, GloballyCleanableResource { private final Map jobs = new HashMap<>(); @@ -84,7 +87,17 @@ public JobManagerJobMetricGroup addJob(JobID jobId, String jobName) { } } - public void removeJob(JobID jobId) { + @Override + public void globalCleanup(JobID jobId) { + cleanup(jobId); + } + + @Override + public void localCleanup(JobID jobId) { + cleanup(jobId); + } + + private void cleanup(JobID jobId) { if (jobId == null) { return; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java index 1a2fa36a38cfc..d4eacb1dad671 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java @@ -142,7 +142,6 @@ public class DefaultScheduler extends SchedulerBase implements SchedulerOperatio jobGraph, ioExecutor, jobMasterConfiguration, - userCodeLoader, checkpointsCleaner, checkpointRecoveryFactory, jobManagerJobMetricGroup, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java index 70d4ab2ac38de..c975085d2d6c3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java @@ -167,7 +167,6 @@ public SchedulerBase( final JobGraph jobGraph, final Executor ioExecutor, final Configuration jobMasterConfiguration, - final ClassLoader userCodeLoader, final CheckpointsCleaner checkpointsCleaner, final CheckpointRecoveryFactory checkpointRecoveryFactory, final JobManagerJobMetricGroup jobManagerJobMetricGroup, @@ -192,7 +191,6 @@ public SchedulerBase( SchedulerUtils.createCompletedCheckpointStoreIfCheckpointingIsEnabled( jobGraph, jobMasterConfiguration, - userCodeLoader, checkNotNull(checkpointRecoveryFactory), ioExecutor, log); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java index 5f6130492262b..f99c51f4970b3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java @@ -20,11 +20,11 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStoreUtils; import org.apache.flink.runtime.checkpoint.DeactivatedCheckpointCompletedCheckpointStore; import org.apache.flink.runtime.checkpoint.DeactivatedCheckpointIDCounter; import org.apache.flink.runtime.client.JobExecutionException; @@ -47,7 +47,6 @@ private SchedulerUtils() { public static CompletedCheckpointStore createCompletedCheckpointStoreIfCheckpointingIsEnabled( JobGraph jobGraph, Configuration configuration, - ClassLoader userCodeLoader, CheckpointRecoveryFactory checkpointRecoveryFactory, Executor ioExecutor, Logger log) @@ -56,12 +55,7 @@ public static CompletedCheckpointStore createCompletedCheckpointStoreIfCheckpoin if (DefaultExecutionGraphBuilder.isCheckpointingEnabled(jobGraph)) { try { return createCompletedCheckpointStore( - configuration, - userCodeLoader, - checkpointRecoveryFactory, - ioExecutor, - log, - jobId); + configuration, checkpointRecoveryFactory, ioExecutor, log, jobId); } catch (Exception e) { throw new JobExecutionException( jobId, @@ -76,32 +70,15 @@ public static CompletedCheckpointStore createCompletedCheckpointStoreIfCheckpoin @VisibleForTesting static CompletedCheckpointStore createCompletedCheckpointStore( Configuration jobManagerConfig, - ClassLoader classLoader, CheckpointRecoveryFactory recoveryFactory, Executor ioExecutor, Logger log, JobID jobId) throws Exception { - int maxNumberOfCheckpointsToRetain = - jobManagerConfig.getInteger(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS); - - if (maxNumberOfCheckpointsToRetain <= 0) { - // warning and use 1 as the default value if the setting in - // state.checkpoints.max-retained-checkpoints is not greater than 0. - log.warn( - "The setting for '{} : {}' is invalid. Using default value of {}", - CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.key(), - maxNumberOfCheckpointsToRetain, - CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.defaultValue()); - - maxNumberOfCheckpointsToRetain = - CheckpointingOptions.MAX_RETAINED_CHECKPOINTS.defaultValue(); - } - return recoveryFactory.createRecoveredCompletedCheckpointStore( jobId, - maxNumberOfCheckpointsToRetain, - classLoader, + CompletedCheckpointStoreUtils.getMaximumNumberOfRetainedCheckpoints( + jobManagerConfig, log), SharedStateRegistry.DEFAULT_FACTORY, ioExecutor); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java index db6c2a17892ba..99410e727326b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java @@ -245,12 +245,7 @@ public AdaptiveScheduler( this.checkpointsCleaner = checkpointsCleaner; this.completedCheckpointStore = SchedulerUtils.createCompletedCheckpointStoreIfCheckpointingIsEnabled( - jobGraph, - configuration, - userCodeClassLoader, - checkpointRecoveryFactory, - ioExecutor, - LOG); + jobGraph, configuration, checkpointRecoveryFactory, ioExecutor, LOG); this.checkpointIdCounter = SchedulerUtils.createCheckpointIDCounterIfCheckpointingIsEnabled( jobGraph, checkpointRecoveryFactory); @@ -759,7 +754,7 @@ private VertexParallelism determineParallelism(SlotAllocator slotAllocator) @Override public ArchivedExecutionGraph getArchivedExecutionGraph( JobStatus jobStatus, @Nullable Throwable cause) { - return ArchivedExecutionGraph.createFromInitializingJob( + return ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobInformation.getJobID(), jobInformation.getName(), jobStatus, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleUtils.java index a25a68ce7846a..663212ed62163 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/NettyShuffleUtils.java @@ -64,15 +64,15 @@ public static Pair getMinMaxNetworkBuffersPerResultPartition( final int sortShuffleMinBuffers, final int numSubpartitions, final ResultPartitionType type) { - int min = - type.isBlocking() && numSubpartitions >= sortShuffleMinParallelism - ? sortShuffleMinBuffers - : numSubpartitions + 1; + boolean isSortShuffle = type.isBlocking() && numSubpartitions >= sortShuffleMinParallelism; + int min = isSortShuffle ? sortShuffleMinBuffers : numSubpartitions + 1; int max = type.isBounded() ? numSubpartitions * configuredNetworkBuffersPerChannel + numFloatingBuffersPerGate - : NetworkBufferPool.UNBOUNDED_POOL_SIZE; + : (isSortShuffle + ? Math.max(min, 4 * numSubpartitions) + : NetworkBufferPool.UNBOUNDED_POOL_SIZE); // for each upstream hash-based blocking/pipelined subpartition, at least one buffer is // needed even the configured network buffers per channel is 0 and this behavior is for // performance. If it's not guaranteed that each subpartition can get at least one buffer, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java index 1d8e4916667c1..d44b385ff68a9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java @@ -26,8 +26,8 @@ import org.apache.flink.configuration.SecurityOptions; import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStoreUtils; import org.apache.flink.runtime.checkpoint.DefaultCompletedCheckpointStore; -import org.apache.flink.runtime.checkpoint.DefaultCompletedCheckpointStoreUtils; import org.apache.flink.runtime.checkpoint.DefaultLastStateConnectionStateListener; import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointIDCounter; import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointStoreUtil; @@ -589,7 +589,7 @@ public static CompletedCheckpointStore createCompletedCheckpoints( final ZooKeeperStateHandleStore completedCheckpointStateHandleStore = createZooKeeperStateHandleStore(client, getCheckpointsPath(), stateStorage); Collection completedCheckpoints = - DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( + CompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( completedCheckpointStateHandleStore, ZooKeeperCheckpointStoreUtil.INSTANCE); final CompletedCheckpointStore zooKeeperCompletedCheckpointStore = new DefaultCompletedCheckpointStore<>( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java index 42fa2353dd573..d06dd9712b0f7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobCacheCleanupTest.java @@ -44,7 +44,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -317,14 +316,12 @@ public void testPermanentBlobDeferredCleanup() throws IOException, InterruptedEx } @Test - public void testTransientBlobNoJobCleanup() - throws IOException, InterruptedException, ExecutionException { + public void testTransientBlobNoJobCleanup() throws Exception { testTransientBlobCleanup(null); } @Test - public void testTransientBlobForJobCleanup() - throws IOException, InterruptedException, ExecutionException { + public void testTransientBlobForJobCleanup() throws Exception { testTransientBlobCleanup(new JobID()); } @@ -332,8 +329,7 @@ public void testTransientBlobForJobCleanup() * Tests that {@link TransientBlobCache} cleans up after a default TTL and keeps files which are * constantly accessed. */ - private void testTransientBlobCleanup(@Nullable final JobID jobId) - throws IOException, InterruptedException, ExecutionException { + private void testTransientBlobCleanup(@Nullable final JobID jobId) throws Exception { // 1s should be a safe-enough buffer to still check for existence after a BLOB's last access long cleanupInterval = 1L; // in seconds @@ -390,7 +386,7 @@ private void testTransientBlobCleanup(@Nullable final JobID jobId) // files are cached now for the given TTL - remove from server so that they are not // re-downloaded if (jobId != null) { - server.cleanupJob(jobId, true); + server.globalCleanup(jobId); } else { server.deleteFromCache(key1); server.deleteFromCache(key2); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java index f4beefd510e8c..c74969c24bbab 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java @@ -27,10 +27,10 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.function.BiConsumerWithException; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import javax.annotation.Nullable; @@ -42,8 +42,10 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Random; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentMap; @@ -59,14 +61,36 @@ import static org.apache.flink.runtime.blob.BlobServerPutTest.verifyContents; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.fail; /** A few tests for the cleanup of transient BLOBs at the {@link BlobServer}. */ public class BlobServerCleanupTest extends TestLogger { - private final Random rnd = new Random(); + private static final Random RANDOM = new Random(); - @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @TempDir private File temporaryFolder; + + private static byte[] createRandomData() { + final byte[] randomData = new byte[2000000]; + RANDOM.nextBytes(randomData); + + return randomData; + } + + private static BlobServer createTestInstance(String storageDirectoryPath, long cleanupInterval) + throws IOException { + return createTestInstance(storageDirectoryPath, cleanupInterval, new VoidBlobStore()); + } + + private static BlobServer createTestInstance( + String storageDirectoryPath, long cleanupInterval, BlobStore blobStore) + throws IOException { + final Configuration config = new Configuration(); + config.setString(BlobServerOptions.STORAGE_DIRECTORY, storageDirectoryPath); + config.setLong(BlobServerOptions.CLEANUP_INTERVAL, cleanupInterval); + + return new BlobServer(config, new File(storageDirectoryPath), blobStore); + } @Test public void testTransientBlobNoJobCleanup() @@ -94,17 +118,13 @@ private void testTransientBlobCleanup(@Nullable final JobID jobId) final List> getOperations = new ArrayList<>(numberConcurrentGetOperations); - byte[] data = new byte[2000000]; - rnd.nextBytes(data); - byte[] data2 = Arrays.copyOfRange(data, 10, 54); - - Configuration config = new Configuration(); - config.setLong(BlobServerOptions.CLEANUP_INTERVAL, cleanupInterval); + byte[] data = createRandomData(); + byte[] data2 = createRandomData(); long cleanupLowerBound; try (BlobServer server = - new BlobServer(config, temporaryFolder.newFolder(), new VoidBlobStore())) { + createTestInstance(temporaryFolder.getAbsolutePath(), cleanupInterval)) { ConcurrentMap, Long> transientBlobExpiryTimes = server.getBlobExpiryTimes(); @@ -136,14 +156,16 @@ private void testTransientBlobCleanup(@Nullable final JobID jobId) final Long key1ExpiryAfterGet = transientBlobExpiryTimes.get(Tuple2.of(jobId, key1)); assertThat(key1ExpiryAfterGet).isGreaterThan(key1ExpiryAfterPut); assertThat(key1ExpiryAfterGet).isGreaterThanOrEqualTo(cleanupLowerBound); - assertEquals(key2ExpiryAfterPut, transientBlobExpiryTimes.get(Tuple2.of(jobId, key2))); + assertThat(key2ExpiryAfterPut) + .isEqualTo(transientBlobExpiryTimes.get(Tuple2.of(jobId, key2))); // access key2, verify expiry times (delay at least 1ms to also verify key1 expiry is // unchanged) Thread.sleep(1); cleanupLowerBound = System.currentTimeMillis() + cleanupInterval; verifyContents(server, jobId, key2, data2); - assertEquals(key1ExpiryAfterGet, transientBlobExpiryTimes.get(Tuple2.of(jobId, key1))); + assertThat(key1ExpiryAfterGet) + .isEqualTo(transientBlobExpiryTimes.get(Tuple2.of(jobId, key1))); assertThat(transientBlobExpiryTimes.get(Tuple2.of(jobId, key2))) .isGreaterThan(key2ExpiryAfterPut); assertThat(transientBlobExpiryTimes.get(Tuple2.of(jobId, key2))) @@ -188,6 +210,139 @@ private void testTransientBlobCleanup(@Nullable final JobID jobId) } } + @Test + public void testLocalCleanup() throws Exception { + final TestingBlobStore blobStore = + createTestingBlobStoreBuilder() + .setDeleteAllFunction( + jobDataToDelete -> + fail( + "No deleteAll call is expected to be triggered but was for %s.", + jobDataToDelete)) + .createTestingBlobStore(); + testSuccessfulCleanup(new JobID(), BlobServer::localCleanup, blobStore); + } + + @Test + public void testGlobalCleanup() throws Exception { + final Set actuallyDeletedJobData = new HashSet<>(); + final JobID jobId = new JobID(); + final TestingBlobStore blobStore = + createTestingBlobStoreBuilder() + .setDeleteAllFunction( + jobDataToDelete -> { + actuallyDeletedJobData.add(jobDataToDelete); + return true; + }) + .createTestingBlobStore(); + testSuccessfulCleanup(jobId, BlobServer::globalCleanup, blobStore); + + assertThat(actuallyDeletedJobData).containsExactlyInAnyOrder(jobId); + } + + @Test + public void testGlobalCleanupUnsuccessfulInBlobStore() throws Exception { + final TestingBlobStore blobStore = + createTestingBlobStoreBuilder() + .setDeleteAllFunction(jobDataToDelete -> false) + .createTestingBlobStore(); + + testFailedCleanup( + new JobID(), + (testInstance, jobId) -> + assertThatThrownBy(() -> testInstance.globalCleanup(new JobID())) + .isInstanceOf(FlinkException.class), + blobStore); + } + + @Test + public void testGlobalCleanupFailureInBlobStore() throws Exception { + final RuntimeException actualException = new RuntimeException("Expected RuntimeException"); + final TestingBlobStore blobStore = + createTestingBlobStoreBuilder() + .setDeleteAllFunction( + jobDataToDelete -> { + throw actualException; + }) + .createTestingBlobStore(); + + testFailedCleanup( + new JobID(), + (testInstance, jobId) -> + assertThatThrownBy(() -> testInstance.globalCleanup(new JobID())) + .isEqualTo(actualException), + blobStore); + } + + private TestingBlobStoreBuilder createTestingBlobStoreBuilder() { + return new TestingBlobStoreBuilder() + .setDeleteFunction( + (jobId, blobKey) -> { + throw new UnsupportedOperationException( + "Deletion of individual blobs is not supported."); + }); + } + + private void testFailedCleanup( + JobID jobId, + BiConsumerWithException callback, + BlobStore blobStore) + throws Exception { + testCleanup(jobId, callback, blobStore, 2); + } + + private void testSuccessfulCleanup( + JobID jobId, + BiConsumerWithException callback, + BlobStore blobStore) + throws Exception { + testCleanup(jobId, callback, blobStore, 0); + } + + private void testCleanup( + JobID jobId, + BiConsumerWithException callback, + BlobStore blobStore, + int expectedFileCountAfterCleanup) + throws Exception { + final JobID otherJobId = new JobID(); + try (BlobServer testInstance = + createTestInstance( + temporaryFolder.getAbsolutePath(), Integer.MAX_VALUE, blobStore)) { + testInstance.start(); + + final BlobKey transientDataBlobKey = + put(testInstance, jobId, createRandomData(), TRANSIENT_BLOB); + final BlobKey otherTransientDataBlobKey = + put(testInstance, otherJobId, createRandomData(), TRANSIENT_BLOB); + + final BlobKey permanentDataBlobKey = + put(testInstance, jobId, createRandomData(), PERMANENT_BLOB); + final BlobKey otherPermanentDataBlobKey = + put(testInstance, otherJobId, createRandomData(), PERMANENT_BLOB); + + checkFilesExist( + jobId, + Arrays.asList(transientDataBlobKey, permanentDataBlobKey), + testInstance, + true); + checkFilesExist( + otherJobId, + Arrays.asList(otherTransientDataBlobKey, otherPermanentDataBlobKey), + testInstance, + true); + + callback.accept(testInstance, jobId); + + checkFileCountForJob(expectedFileCountAfterCleanup, jobId, testInstance); + checkFilesExist( + otherJobId, + Arrays.asList(otherTransientDataBlobKey, otherPermanentDataBlobKey), + testInstance, + true); + } + } + @Test public void testBlobServerExpiresRecoveredTransientJobBlob() throws Exception { runBlobServerExpiresRecoveredTransientBlob(new JobID()); @@ -201,17 +356,14 @@ public void testBlobServerExpiresRecoveredTransientNoJobBlob() throws Exception private void runBlobServerExpiresRecoveredTransientBlob(@Nullable JobID jobId) throws Exception { final long cleanupInterval = 1L; - final Configuration configuration = new Configuration(); - configuration.set(BlobServerOptions.CLEANUP_INTERVAL, cleanupInterval); - final File storageDirectory = temporaryFolder.newFolder(); final TransientBlobKey transientBlobKey = TestingBlobUtils.writeTransientBlob( - storageDirectory.toPath(), jobId, new byte[] {1, 2, 3, 4}); - final File blob = BlobUtils.getStorageLocation(storageDirectory, jobId, transientBlobKey); + temporaryFolder.toPath(), jobId, new byte[] {1, 2, 3, 4}); + final File blob = BlobUtils.getStorageLocation(temporaryFolder, jobId, transientBlobKey); try (final BlobServer blobServer = - new BlobServer(configuration, storageDirectory, new VoidBlobStore())) { + createTestInstance(temporaryFolder.getAbsolutePath(), cleanupInterval)) { CommonTestUtils.waitUntilCondition( () -> !blob.exists(), Deadline.fromNow(Duration.ofSeconds(cleanupInterval * 5L)), @@ -220,20 +372,20 @@ private void runBlobServerExpiresRecoveredTransientBlob(@Nullable JobID jobId) } @Test - public void testBlobServerRetainsJobs() throws IOException { - final File storageDirectory = temporaryFolder.newFolder(); - + public void testBlobServerRetainsJobs() throws Exception { final JobID jobId1 = new JobID(); final JobID jobId2 = new JobID(); final byte[] fileContent = {1, 2, 3, 4}; final PermanentBlobKey blobKey1 = - TestingBlobUtils.writePermanentBlob(storageDirectory.toPath(), jobId1, fileContent); + TestingBlobUtils.writePermanentBlob(temporaryFolder.toPath(), jobId1, fileContent); final PermanentBlobKey blobKey2 = - TestingBlobUtils.writePermanentBlob(storageDirectory.toPath(), jobId2, fileContent); + TestingBlobUtils.writePermanentBlob(temporaryFolder.toPath(), jobId2, fileContent); try (final BlobServer blobServer = - new BlobServer(new Configuration(), storageDirectory, new VoidBlobStore())) { + createTestInstance( + temporaryFolder.getAbsolutePath(), + BlobServerOptions.CLEANUP_INTERVAL.defaultValue())) { blobServer.retainJobs(Collections.singleton(jobId1)); assertThat(blobServer.getFile(jobId1, blobKey1)).hasBinaryContent(fileContent); @@ -314,10 +466,9 @@ public static void checkFileCountForJob( throw new IOException("File " + jobDir + " does not exist."); } } else { - assertEquals( - "Too many/few files in job dir: " + Arrays.asList(blobsForJob).toString(), - expectedCount, - blobsForJob.length); + assertThat(blobsForJob.length) + .as("Too many/few files in job dir: " + Arrays.asList(blobsForJob)) + .isEqualTo(expectedCount); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java index a87a2b1c4cd06..f6acef3f38ec0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerDeleteTest.java @@ -266,21 +266,22 @@ private void testDeleteBlobFails(@Nullable final JobID jobId, BlobKey.BlobType b } @Test - public void testJobCleanup() throws IOException { + public void testJobCleanup() throws Exception { testJobCleanup(TRANSIENT_BLOB); } @Test - public void testJobCleanupHa() throws IOException { + public void testJobCleanupHa() throws Exception { testJobCleanup(PERMANENT_BLOB); } /** - * Tests that {@link BlobServer} cleans up after calling {@link BlobServer#cleanupJob}. + * Tests that {@link BlobServer} cleans up after calling {@link + * BlobServer#globalCleanup(JobID)}. * * @param blobType whether the BLOB should become permanent or transient */ - private void testJobCleanup(BlobKey.BlobType blobType) throws IOException { + private void testJobCleanup(BlobKey.BlobType blobType) throws Exception { JobID jobId1 = new JobID(); JobID jobId2 = new JobID(); @@ -308,7 +309,7 @@ private void testJobCleanup(BlobKey.BlobType blobType) throws IOException { verifyContents(server, jobId2, key2, data); checkFileCountForJob(1, jobId2, server); - server.cleanupJob(jobId1, true); + server.globalCleanup(jobId1); verifyDeleted(server, jobId1, key1a); verifyDeleted(server, jobId1, key1b); @@ -316,14 +317,14 @@ private void testJobCleanup(BlobKey.BlobType blobType) throws IOException { verifyContents(server, jobId2, key2, data); checkFileCountForJob(1, jobId2, server); - server.cleanupJob(jobId2, true); + server.globalCleanup(jobId2); checkFileCountForJob(0, jobId1, server); verifyDeleted(server, jobId2, key2); checkFileCountForJob(0, jobId2, server); // calling a second time should not fail - server.cleanupJob(jobId2, true); + server.globalCleanup(jobId2); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRecoveryTest.java index dfa279930638e..655d64adb50a7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRecoveryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerRecoveryTest.java @@ -89,7 +89,7 @@ public void testBlobServerRecovery() throws Exception { */ public static void testBlobServerRecovery( final Configuration config, final BlobStore blobStore, final File blobStorage) - throws IOException { + throws Exception { final String clusterId = config.getString(HighAvailabilityOptions.HA_CLUSTER_ID); String storagePath = config.getString(HighAvailabilityOptions.HA_STORAGE_PATH) + "/" + clusterId; @@ -141,8 +141,8 @@ public static void testBlobServerRecovery( verifyDeleted(cache1, jobId[0], nonHAKey); // Remove again - server1.cleanupJob(jobId[0], true); - server1.cleanupJob(jobId[1], true); + server1.globalCleanup(jobId[0]); + server1.globalCleanup(jobId[1]); // Verify everything is clean assertTrue("HA storage directory does not exist", fs.exists(new Path(storagePath))); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/FileSystemBlobStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/FileSystemBlobStoreTest.java index 5b72270d63d78..479f58475fcd7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/FileSystemBlobStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/FileSystemBlobStoreTest.java @@ -22,26 +22,255 @@ import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.core.fs.local.LocalDataOutputStream; import org.apache.flink.runtime.state.filesystem.TestFs; +import org.apache.flink.testutils.TestFileSystem; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.function.FunctionWithException; import org.apache.commons.io.FileUtils; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.io.TempDir; +import java.io.BufferedWriter; import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileWriter; import java.io.IOException; +import java.io.InputStream; import java.nio.charset.StandardCharsets; +import java.nio.file.Files; import java.nio.file.Path; +import java.security.MessageDigest; import java.util.concurrent.atomic.AtomicReference; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the {@link FileSystemBlobStore}. */ @ExtendWith(TestLoggerExtension.class) class FileSystemBlobStoreTest { + private FileSystemBlobStore testInstance; + private Path storagePath; + + @BeforeEach + public void createTestInstance(@TempDir Path storagePath) throws IOException { + this.testInstance = new FileSystemBlobStore(new TestFileSystem(), storagePath.toString()); + this.storagePath = storagePath; + } + + @AfterEach + public void finalizeTestInstance() throws IOException { + testInstance.close(); + } + + @Test + public void testSuccessfulPut() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("put"); + + final JobID jobId = new JobID(); + final BlobKey blobKey = createPermanentBlobKeyFromFile(temporaryFile); + assertThat(getBlobDirectoryPath()).isEmptyDirectory(); + + final boolean successfullyWritten = + testInstance.put(temporaryFile.toFile(), jobId, blobKey); + assertThat(successfullyWritten).isTrue(); + + assertThat(getPath(jobId)).isDirectory().exists(); + assertThat(getPath(jobId, blobKey)).isNotEmptyFile().hasSameTextualContentAs(temporaryFile); + } + + @Test + public void testMissingFilePut() throws IOException { + assertThatThrownBy( + () -> + testInstance.put( + new File("/not/existing/file"), + new JobID(), + new PermanentBlobKey())) + .isInstanceOf(FileNotFoundException.class); + } + + @Test + public void testSuccessfulGet() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("get"); + final JobID jobId = new JobID(); + final BlobKey blobKey = createPermanentBlobKeyFromFile(temporaryFile); + + assertThat(testInstance.put(temporaryFile.toFile(), jobId, blobKey)).isTrue(); + + final Path targetFile = Files.createTempFile("filesystemblobstoretest-get-target-", ""); + assertThat(targetFile).isEmptyFile(); + final boolean successfullyGet = testInstance.get(jobId, blobKey, targetFile.toFile()); + assertThat(successfullyGet).isTrue(); + + assertThat(targetFile).hasSameTextualContentAs(temporaryFile); + } + + @Test + public void testGetWithWrongJobId() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("get"); + final BlobKey blobKey = createPermanentBlobKeyFromFile(temporaryFile); + + assertThat(testInstance.put(temporaryFile.toFile(), new JobID(), blobKey)).isTrue(); + + assertThatThrownBy( + () -> + testInstance.get( + new JobID(), + blobKey, + Files.createTempFile( + "filesystemblobstoretest-get-with-wrong-jobid-", + "") + .toFile())) + .isInstanceOf(FileNotFoundException.class); + } + + @Test + public void testGetWithWrongBlobKey() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("get"); + + final JobID jobId = new JobID(); + assertThat(testInstance.put(temporaryFile.toFile(), jobId, new PermanentBlobKey())) + .isTrue(); + + assertThatThrownBy( + () -> + testInstance.get( + jobId, + new PermanentBlobKey(), + Files.createTempFile( + "filesystemblobstoretest-get-with-wrong-blobkey-", + "") + .toFile())) + .isInstanceOf(FileNotFoundException.class); + } + + @Test + public void testSuccessfulDeleteOnlyBlob() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("delete"); + final JobID jobId = new JobID(); + final BlobKey blobKey = createPermanentBlobKeyFromFile(temporaryFile); + + assertThat(testInstance.put(temporaryFile.toFile(), jobId, blobKey)).isTrue(); + + assertThat(getPath(jobId)).isDirectory().exists(); + assertThat(getPath(jobId, blobKey)).isNotEmptyFile(); + + final boolean successfullyDeleted = testInstance.delete(jobId, blobKey); + + assertThat(successfullyDeleted).isTrue(); + assertThat(getPath(jobId)).doesNotExist(); + } + + @Test + public void testSuccessfulDeleteBlob() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("delete"); + final JobID jobId = new JobID(); + final BlobKey blobKey = createPermanentBlobKeyFromFile(temporaryFile); + final BlobKey otherBlobKey = new PermanentBlobKey(); + + assertThat(testInstance.put(temporaryFile.toFile(), jobId, blobKey)).isTrue(); + // create another artifact to omit deleting the directory + assertThat(testInstance.put(temporaryFile.toFile(), jobId, otherBlobKey)).isTrue(); + + assertThat(getPath(jobId)).isDirectory().exists(); + assertThat(getPath(jobId, blobKey)).isNotEmptyFile(); + assertThat(getPath(jobId, otherBlobKey)).isNotEmptyFile(); + + final boolean successfullyDeleted = testInstance.delete(jobId, blobKey); + + assertThat(successfullyDeleted).isTrue(); + assertThat(getPath(jobId, otherBlobKey)).exists(); + } + + @Test + public void testDeleteWithNotExistingJobId() { + assertThat(testInstance.delete(new JobID(), new PermanentBlobKey())).isTrue(); + } + + @Test + public void testDeleteWithNotExistingBlobKey() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("delete"); + final JobID jobId = new JobID(); + final BlobKey blobKey = createPermanentBlobKeyFromFile(temporaryFile); + + assertThat(testInstance.put(temporaryFile.toFile(), jobId, blobKey)).isTrue(); + assertThat(testInstance.delete(jobId, new PermanentBlobKey())).isTrue(); + assertThat(getPath(jobId, blobKey)).exists(); + } + + @Test + public void testDeleteAll() throws IOException { + final Path temporaryFile = createTemporaryFileWithContent("delete"); + final JobID jobId = new JobID(); + + assertThat(testInstance.put(temporaryFile.toFile(), jobId, new PermanentBlobKey())) + .isTrue(); + assertThat(testInstance.put(temporaryFile.toFile(), jobId, new PermanentBlobKey())) + .isTrue(); + + assertThat(getPath(jobId)).isDirectory().exists(); + assertThat(getPath(jobId).toFile().listFiles()).hasSize(2); + + assertThat(testInstance.deleteAll(jobId)).isTrue(); + assertThat(getPath(jobId)).doesNotExist(); + } + + @Test + public void testDeleteAllWithNotExistingJobId() { + final JobID jobId = new JobID(); + assertThat(testInstance.deleteAll(jobId)).isTrue(); + assertThat(getPath(jobId)).doesNotExist(); + } + + private Path createTemporaryFileWithContent(String operationLabel) throws IOException { + final String actualContent = + String.format("Content for testing the %s operation", operationLabel); + final Path temporaryFile = + Files.createTempFile( + String.format("filesystemblobstoretest-%s-", operationLabel), ""); + try (BufferedWriter writer = + new BufferedWriter(new FileWriter(temporaryFile.toAbsolutePath().toString()))) { + writer.write(actualContent); + } + + return temporaryFile; + } + + private Path getBlobDirectoryPath() { + return storagePath.resolve(FileSystemBlobStore.BLOB_PATH_NAME); + } + + private Path getPath(JobID jobId) { + return getBlobDirectoryPath().resolve(String.format("job_%s", jobId)); + } + + private Path getPath(JobID jobId, BlobKey blobKey) { + return getPath(jobId).resolve(String.format("blob_%s", blobKey)); + } + + private BlobKey createPermanentBlobKeyFromFile(Path path) throws IOException { + Preconditions.checkArgument(!Files.isDirectory(path)); + Preconditions.checkArgument(Files.exists(path)); + + MessageDigest md = BlobUtils.createMessageDigest(); + try (InputStream is = new FileInputStream(path.toFile())) { + final byte[] buf = new byte[1024]; + int bytesRead = is.read(buf); + while (bytesRead >= 0) { + md.update(buf, 0, bytesRead); + bytesRead = is.read(buf); + } + + return BlobKey.createKey(BlobKey.BlobType.PERMANENT_BLOB, md.digest()); + } + } + @Test public void fileSystemBlobStoreCallsSyncOnPut(@TempDir Path storageDirectory) throws IOException { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreUtilsTest.java similarity index 82% rename from flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreUtilsTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreUtilsTest.java index 984f1059dc0be..77481aed1ff4d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointStoreUtilsTest.java @@ -20,6 +20,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.persistence.TestingStateHandleStore; import org.apache.flink.runtime.state.RetrievableStateHandle; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; @@ -27,6 +29,8 @@ import org.apache.flink.util.TestLogger; import org.junit.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import java.io.IOException; import java.io.Serializable; @@ -38,11 +42,12 @@ import java.util.List; import java.util.stream.Collectors; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; -/** Tests related to {@link DefaultCompletedCheckpointStoreUtils}. */ -public class DefaultCompletedCheckpointStoreUtilsTest extends TestLogger { +/** Tests related to {@link CompletedCheckpointStoreUtils}. */ +public class CompletedCheckpointStoreUtilsTest extends TestLogger { private static CompletedCheckpoint createCompletedCheckpoint(long checkpointId) { return new CompletedCheckpoint( @@ -105,7 +110,7 @@ public void testRetrievedCheckpointsAreOrderedChronologically() throws Exception .setGetAllSupplier(() -> handles) .build(); final Collection completedCheckpoints = - DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( + CompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( stateHandleStore, new SimpleCheckpointStoreUtil()); // Make sure checkpoints are ordered from earliest to latest. assertEquals( @@ -132,7 +137,19 @@ public void testRetrievingCheckpointsFailsIfRetrievalOfAnyCheckpointFails() thro assertThrows( FlinkException.class, () -> - DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( + CompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( stateHandleStore, new SimpleCheckpointStoreUtil())); } + + @ParameterizedTest(name = "actual: {0}; expected: {1}") + @CsvSource({"10,10", "0,1", "-1,1"}) + public void testGetMaximumNumberOfRetainedCheckpoints(int actualValue, int expectedValue) { + final Configuration jobManagerConfig = new Configuration(); + jobManagerConfig.setInteger(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS, actualValue); + + assertThat( + CompletedCheckpointStoreUtils.getMaximumNumberOfRetainedCheckpoints( + jobManagerConfig, log)) + .isEqualTo(expectedValue); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java index d7ea8804584f4..0447cc172b1d2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java @@ -139,7 +139,7 @@ private void testCheckpointRetention( /** * We have three completed checkpoints(1, 2, 3) in the state handle store. We expect that {@link - * DefaultCompletedCheckpointStoreUtils#retrieveCompletedCheckpoints(StateHandleStore, + * CompletedCheckpointStoreUtils#retrieveCompletedCheckpoints(StateHandleStore, * CheckpointStoreUtil)} should recover the sorted checkpoints by name. */ @Test @@ -390,7 +390,7 @@ public long nameToCheckpointID(String name) { toRetain, stateHandleStore, checkpointStoreUtil, - DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( + CompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( stateHandleStore, checkpointStoreUtil), SharedStateRegistry.DEFAULT_FACTORY.create( org.apache.flink.util.concurrent.Executors.directExecutor(), emptyList()), 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 190487e973c9a..7676bf49e0f73 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 @@ -56,10 +56,14 @@ public class DefaultSchedulerCheckpointCoordinatorTest extends TestLogger { public void testClosingSchedulerShutsDownCheckpointCoordinatorOnFailedExecutionGraph() throws Exception { final CompletableFuture counterShutdownFuture = new CompletableFuture<>(); - CheckpointIDCounter counter = new TestingCheckpointIDCounter(counterShutdownFuture); + CheckpointIDCounter counter = + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + counterShutdownFuture); final CompletableFuture storeShutdownFuture = new CompletableFuture<>(); - CompletedCheckpointStore store = new TestingCompletedCheckpointStore(storeShutdownFuture); + CompletedCheckpointStore store = + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints(storeShutdownFuture); final SchedulerBase scheduler = createSchedulerAndEnableCheckpointing(counter, store); final ExecutionGraph graph = scheduler.getExecutionGraph(); @@ -82,10 +86,14 @@ public void testClosingSchedulerShutsDownCheckpointCoordinatorOnFailedExecutionG public void testClosingSchedulerShutsDownCheckpointCoordinatorOnSuspendedExecutionGraph() throws Exception { final CompletableFuture counterShutdownFuture = new CompletableFuture<>(); - CheckpointIDCounter counter = new TestingCheckpointIDCounter(counterShutdownFuture); + CheckpointIDCounter counter = + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + counterShutdownFuture); final CompletableFuture storeShutdownFuture = new CompletableFuture<>(); - CompletedCheckpointStore store = new TestingCompletedCheckpointStore(storeShutdownFuture); + CompletedCheckpointStore store = + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints(storeShutdownFuture); final SchedulerBase scheduler = createSchedulerAndEnableCheckpointing(counter, store); final ExecutionGraph graph = scheduler.getExecutionGraph(); @@ -108,10 +116,14 @@ public void testClosingSchedulerShutsDownCheckpointCoordinatorOnSuspendedExecuti public void testClosingSchedulerShutsDownCheckpointCoordinatorOnFinishedExecutionGraph() throws Exception { final CompletableFuture counterShutdownFuture = new CompletableFuture<>(); - CheckpointIDCounter counter = new TestingCheckpointIDCounter(counterShutdownFuture); + CheckpointIDCounter counter = + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + counterShutdownFuture); final CompletableFuture storeShutdownFuture = new CompletableFuture<>(); - CompletedCheckpointStore store = new TestingCompletedCheckpointStore(storeShutdownFuture); + CompletedCheckpointStore store = + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints(storeShutdownFuture); final SchedulerBase scheduler = createSchedulerAndEnableCheckpointing(counter, store); final ExecutionGraph graph = scheduler.getExecutionGraph(); @@ -143,10 +155,14 @@ public void testClosingSchedulerShutsDownCheckpointCoordinatorOnFinishedExecutio public void testClosingSchedulerSuspendsExecutionGraphAndShutsDownCheckpointCoordinator() throws Exception { final CompletableFuture counterShutdownFuture = new CompletableFuture<>(); - CheckpointIDCounter counter = new TestingCheckpointIDCounter(counterShutdownFuture); + CheckpointIDCounter counter = + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + counterShutdownFuture); final CompletableFuture storeShutdownFuture = new CompletableFuture<>(); - CompletedCheckpointStore store = new TestingCompletedCheckpointStore(storeShutdownFuture); + CompletedCheckpointStore store = + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints(storeShutdownFuture); final SchedulerBase scheduler = createSchedulerAndEnableCheckpointing(counter, store); final ExecutionGraph graph = scheduler.getExecutionGraph(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java index acd9d809f4d28..58a0d954318dd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java @@ -36,19 +36,18 @@ public class PerJobCheckpointRecoveryTest extends TestLogger { @Test public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { final TestingCompletedCheckpointStore store = - new TestingCompletedCheckpointStore(new CompletableFuture<>()); + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints( + new CompletableFuture<>()); final CheckpointRecoveryFactory factory = PerJobCheckpointRecoveryFactory.withoutCheckpointStoreRecovery( maxCheckpoints -> store); - final ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); - final JobID firstJobId = new JobID(); assertSame( store, factory.createRecoveredCompletedCheckpointStore( firstJobId, 1, - classLoader, SharedStateRegistry.DEFAULT_FACTORY, Executors.directExecutor())); assertThrows( @@ -57,7 +56,6 @@ public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { factory.createRecoveredCompletedCheckpointStore( firstJobId, 1, - classLoader, SharedStateRegistry.DEFAULT_FACTORY, Executors.directExecutor())); @@ -67,7 +65,6 @@ public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { factory.createRecoveredCompletedCheckpointStore( secondJobId, 1, - classLoader, SharedStateRegistry.DEFAULT_FACTORY, Executors.directExecutor())); assertThrows( @@ -76,7 +73,6 @@ public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { factory.createRecoveredCompletedCheckpointStore( secondJobId, 1, - classLoader, SharedStateRegistry.DEFAULT_FACTORY, Executors.directExecutor())); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java index 79df955d121f1..2365ed59789cf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java @@ -20,36 +20,109 @@ import org.apache.flink.api.common.JobStatus; import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; +import java.util.function.Supplier; /** Test {@link CheckpointIDCounter} implementation for testing the shutdown behavior. */ public final class TestingCheckpointIDCounter implements CheckpointIDCounter { - private final CompletableFuture shutdownStatus; + private final Runnable startRunnable; + private final Consumer shutdownConsumer; + private final Supplier getAndIncrementSupplier; + private final Supplier getSupplier; + private final Consumer setCountConsumer; - public TestingCheckpointIDCounter(CompletableFuture shutdownStatus) { - this.shutdownStatus = shutdownStatus; + public static TestingCheckpointIDCounter createStoreWithShutdownCheckAndNoStartAction( + CompletableFuture shutdownFuture) { + return TestingCheckpointIDCounter.builder() + .withStartRunnable(() -> {}) + .withShutdownConsumer(shutdownFuture::complete) + .build(); + } + + private TestingCheckpointIDCounter( + Runnable startRunnable, + Consumer shutdownConsumer, + Supplier getAndIncrementSupplier, + Supplier getSupplier, + Consumer setCountConsumer) { + this.startRunnable = startRunnable; + this.shutdownConsumer = shutdownConsumer; + this.getAndIncrementSupplier = getAndIncrementSupplier; + this.getSupplier = getSupplier; + this.setCountConsumer = setCountConsumer; } @Override - public void start() {} + public void start() { + startRunnable.run(); + } @Override public void shutdown(JobStatus jobStatus) { - shutdownStatus.complete(jobStatus); + shutdownConsumer.accept(jobStatus); } @Override public long getAndIncrement() { - throw new UnsupportedOperationException("Not implemented."); + return getAndIncrementSupplier.get(); } @Override public long get() { - throw new UnsupportedOperationException("Not implemented."); + return getSupplier.get(); } @Override public void setCount(long newId) { - throw new UnsupportedOperationException("Not implemented."); + setCountConsumer.accept(newId); + } + + public static Builder builder() { + return new Builder(); + } + + /** {@code Builder} for creating {@code TestingCheckpointIDCounter} instances. */ + public static class Builder { + + private Runnable startRunnable; + private Consumer shutdownConsumer; + private Supplier getAndIncrementSupplier; + private Supplier getSupplier; + private Consumer setCountConsumer; + + public Builder withStartRunnable(Runnable startRunnable) { + this.startRunnable = startRunnable; + return this; + } + + public Builder withShutdownConsumer(Consumer shutdownConsumer) { + this.shutdownConsumer = shutdownConsumer; + return this; + } + + public Builder withGetAndIncrementSupplier(Supplier getAndIncrementSupplier) { + this.getAndIncrementSupplier = getAndIncrementSupplier; + return this; + } + + public Builder withGetSupplier(Supplier getSupplier) { + this.getSupplier = getSupplier; + return this; + } + + public Builder withSetCountConsumer(Consumer setCountConsumer) { + this.setCountConsumer = setCountConsumer; + return this; + } + + public TestingCheckpointIDCounter build() { + return new TestingCheckpointIDCounter( + startRunnable, + shutdownConsumer, + getAndIncrementSupplier, + getSupplier, + setCountConsumer); + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java index c1e851478b130..e164543d0ab81 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java @@ -38,7 +38,6 @@ public TestingCheckpointRecoveryFactory( public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, - ClassLoader userClassLoader, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor) { return store; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCompletedCheckpointStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCompletedCheckpointStore.java index dc9a1922a3cc4..1d474a54b3cce 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCompletedCheckpointStore.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCompletedCheckpointStore.java @@ -19,18 +19,54 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.util.function.TriFunction; import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.function.BiConsumer; +import java.util.function.Supplier; /** Test {@link CompletedCheckpointStore} implementation for testing the shutdown behavior. */ public final class TestingCompletedCheckpointStore implements CompletedCheckpointStore { - private final CompletableFuture shutdownStatus; + private final TriFunction< + CompletedCheckpoint, CheckpointsCleaner, Runnable, CompletedCheckpoint> + addCheckpointAndSubsumeOldestOneFunction; + private final BiConsumer shutdownConsumer; + private final Supplier> getAllCheckpointsSupplier; + private final Supplier getNumberOfRetainedCheckpointsSuppler; + private final Supplier getMaxNumberOfRetainedCheckpointsSupplier; + private final Supplier requiresExternalizedCheckpointsSupplier; + private final Supplier getSharedStateRegistrySupplier; - public TestingCompletedCheckpointStore(CompletableFuture shutdownStatus) { - this.shutdownStatus = shutdownStatus; + public static TestingCompletedCheckpointStore + createStoreWithShutdownCheckAndNoCompletedCheckpoints( + CompletableFuture shutdownFuture) { + return TestingCompletedCheckpointStore.builder() + .withShutdownConsumer( + ((jobStatus, ignoredCheckpointsCleaner) -> + shutdownFuture.complete(jobStatus))) + .withGetAllCheckpointsSupplier(Collections::emptyList) + .build(); + } + + private TestingCompletedCheckpointStore( + TriFunction + addCheckpointAndSubsumeOldestOneFunction, + BiConsumer shutdownConsumer, + Supplier> getAllCheckpointsSupplier, + Supplier getNumberOfRetainedCheckpointsSuppler, + Supplier getMaxNumberOfRetainedCheckpointsSupplier, + Supplier requiresExternalizedCheckpointsSupplier, + Supplier getSharedStateRegistrySupplier) { + this.addCheckpointAndSubsumeOldestOneFunction = addCheckpointAndSubsumeOldestOneFunction; + this.shutdownConsumer = shutdownConsumer; + this.getAllCheckpointsSupplier = getAllCheckpointsSupplier; + this.getNumberOfRetainedCheckpointsSuppler = getNumberOfRetainedCheckpointsSuppler; + this.getMaxNumberOfRetainedCheckpointsSupplier = getMaxNumberOfRetainedCheckpointsSupplier; + this.requiresExternalizedCheckpointsSupplier = requiresExternalizedCheckpointsSupplier; + this.getSharedStateRegistrySupplier = getSharedStateRegistrySupplier; } @Override @@ -38,41 +74,138 @@ public CompletedCheckpoint addCheckpointAndSubsumeOldestOne( CompletedCheckpoint checkpoint, CheckpointsCleaner checkpointsCleaner, Runnable postCleanup) { - throw new UnsupportedOperationException("Not implemented."); - } - - @Override - public CompletedCheckpoint getLatestCheckpoint() { - return null; + return addCheckpointAndSubsumeOldestOneFunction.apply( + checkpoint, checkpointsCleaner, postCleanup); } @Override public void shutdown(JobStatus jobStatus, CheckpointsCleaner checkpointsCleaner) { - shutdownStatus.complete(jobStatus); + shutdownConsumer.accept(jobStatus, checkpointsCleaner); } @Override public List getAllCheckpoints() { - return Collections.emptyList(); + return getAllCheckpointsSupplier.get(); } @Override public int getNumberOfRetainedCheckpoints() { - throw new UnsupportedOperationException("Not implemented."); + return getNumberOfRetainedCheckpointsSuppler.get(); } @Override public int getMaxNumberOfRetainedCheckpoints() { - throw new UnsupportedOperationException("Not implemented."); + return getMaxNumberOfRetainedCheckpointsSupplier.get(); } @Override public boolean requiresExternalizedCheckpoints() { - throw new UnsupportedOperationException("Not implemented."); + return requiresExternalizedCheckpointsSupplier.get(); } @Override public SharedStateRegistry getSharedStateRegistry() { - throw new UnsupportedOperationException("Not implemented."); + return getSharedStateRegistrySupplier.get(); + } + + public static Builder builder() { + return new TestingCompletedCheckpointStore.Builder(); + } + + /** {@code Builder} for creating {@code TestingCompletedCheckpointStore} instances. */ + public static class Builder { + + private TriFunction + addCheckpointAndSubsumeOldestOneFunction = + (ignoredCompletedCheckpoint, + ignoredCheckpointsCleaner, + ignoredPostCleanup) -> { + throw new UnsupportedOperationException( + "addCheckpointAndSubsumeOldestOne is not implemented."); + }; + private BiConsumer shutdownConsumer = + (ignoredJobStatus, ignoredCheckpointsCleaner) -> { + throw new UnsupportedOperationException("shutdown is not implemented."); + }; + private Supplier> getAllCheckpointsSupplier = + () -> { + throw new UnsupportedOperationException( + "getAllCheckpoints is not implemented."); + }; + private Supplier getNumberOfRetainedCheckpointsSuppler = + () -> { + throw new UnsupportedOperationException( + "getNumberOfRetainedCheckpointsis not implemented."); + }; + private Supplier getMaxNumberOfRetainedCheckpointsSupplier = + () -> { + throw new UnsupportedOperationException( + "getMaxNumberOfRetainedCheckpoints is not implemented."); + }; + private Supplier requiresExternalizedCheckpointsSupplier = + () -> { + throw new UnsupportedOperationException( + "requiresExternalizedCheckpoints is not implemented."); + }; + private Supplier getSharedStateRegistrySupplier = + () -> { + throw new UnsupportedOperationException( + "getSharedStateRegistry is not implemented."); + }; + + public Builder withAddCheckpointAndSubsumeOldestOneFunction( + TriFunction + addCheckpointAndSubsumeOldestOneFunction) { + this.addCheckpointAndSubsumeOldestOneFunction = + addCheckpointAndSubsumeOldestOneFunction; + return this; + } + + public Builder withShutdownConsumer( + BiConsumer shutdownConsumer) { + this.shutdownConsumer = shutdownConsumer; + return this; + } + + public Builder withGetAllCheckpointsSupplier( + Supplier> getAllCheckpointsSupplier) { + this.getAllCheckpointsSupplier = getAllCheckpointsSupplier; + return this; + } + + public Builder withGetNumberOfRetainedCheckpointsSupplier( + Supplier getNumberOfRetainedCheckpointsSuppler) { + this.getNumberOfRetainedCheckpointsSuppler = getNumberOfRetainedCheckpointsSuppler; + return this; + } + + public Builder withGetMaxNumberOfRetainedCheckpointsSupplier( + Supplier getMaxNumberOfRetainedCheckpoints) { + this.getMaxNumberOfRetainedCheckpointsSupplier = getMaxNumberOfRetainedCheckpoints; + return this; + } + + public Builder withRequiresExternalizedCheckpointsSupplier( + Supplier requiresExternalizedCheckpointsSupplier) { + this.requiresExternalizedCheckpointsSupplier = requiresExternalizedCheckpointsSupplier; + return this; + } + + public Builder withGetSharedStateRegistrySupplier( + Supplier getSharedStateRegistrySupplier) { + this.getSharedStateRegistrySupplier = getSharedStateRegistrySupplier; + return this; + } + + public TestingCompletedCheckpointStore build() { + return new TestingCompletedCheckpointStore( + addCheckpointAndSubsumeOldestOneFunction, + shutdownConsumer, + getAllCheckpointsSupplier, + getNumberOfRetainedCheckpointsSuppler, + getMaxNumberOfRetainedCheckpointsSupplier, + requiresExternalizedCheckpointsSupplier, + getSharedStateRegistrySupplier); + } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java index 65a435b6fc8cd..2deea44ff591e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java @@ -89,7 +89,7 @@ protected CompletedCheckpointStore createRecoveredCompletedCheckpointStore( maxNumberOfCheckpointsToRetain, checkpointsInZooKeeper, checkpointStoreUtil, - DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( + CompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( checkpointsInZooKeeper, checkpointStoreUtil), SharedStateRegistry.DEFAULT_FACTORY.create(Executors.directExecutor(), emptyList()), executor); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java index e56f0a38b563c..3aa7c2ddc4d84 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java @@ -104,7 +104,7 @@ public void testRecoverFailsIfDownloadFails() { assertThrows( Exception.class, () -> - DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( + CompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( checkpointsInZooKeeper, zooKeeperCheckpointStoreUtil)); assertThat(exception, FlinkMatchers.containsCause(ExpectedTestException.class)); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ApplicationStatusTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ApplicationStatusTest.java index 9c164c2bd4fcf..6cc225d0001c5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ApplicationStatusTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ApplicationStatusTest.java @@ -18,41 +18,104 @@ package org.apache.flink.runtime.clusterframework; -import org.apache.flink.util.TestLogger; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.util.TestLoggerExtension; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import java.util.Arrays; import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import static org.hamcrest.Matchers.contains; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.not; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the {@link ApplicationStatus}. */ -public class ApplicationStatusTest extends TestLogger { +@ExtendWith(TestLoggerExtension.class) +public class ApplicationStatusTest { private static final int SUCCESS_EXIT_CODE = 0; @Test public void succeededStatusMapsToSuccessExitCode() { int exitCode = ApplicationStatus.SUCCEEDED.processExitCode(); - assertThat(exitCode, is(equalTo(SUCCESS_EXIT_CODE))); + assertThat(exitCode).isEqualTo(SUCCESS_EXIT_CODE); } @Test public void cancelledStatusMapsToSuccessExitCode() { int exitCode = ApplicationStatus.CANCELED.processExitCode(); - assertThat(exitCode, is(equalTo(SUCCESS_EXIT_CODE))); + assertThat(exitCode).isEqualTo(SUCCESS_EXIT_CODE); } @Test public void notSucceededNorCancelledStatusMapsToNonSuccessExitCode() { Iterable exitCodes = exitCodes(notSucceededNorCancelledStatus()); - assertThat(exitCodes, not(contains(SUCCESS_EXIT_CODE))); + assertThat(exitCodes).doesNotContain(SUCCESS_EXIT_CODE); + } + + @Test + public void testJobStatusFromSuccessApplicationStatus() { + assertThat(ApplicationStatus.SUCCEEDED.deriveJobStatus()).isEqualTo(JobStatus.FINISHED); + } + + @Test + public void testJobStatusFromFailedApplicationStatus() { + assertThat(ApplicationStatus.FAILED.deriveJobStatus()).isEqualTo(JobStatus.FAILED); + } + + @Test + public void testJobStatusFromCancelledApplicationStatus() { + assertThat(ApplicationStatus.CANCELED.deriveJobStatus()).isEqualTo(JobStatus.CANCELED); + } + + @Test + public void testJobStatusFailsFromUnknownApplicationStatuses() { + assertThatThrownBy(ApplicationStatus.UNKNOWN::deriveJobStatus) + .isInstanceOf(UnsupportedOperationException.class); + } + + @Test + public void testSuccessApplicationStatusFromJobStatus() { + assertThat(ApplicationStatus.fromJobStatus(JobStatus.FINISHED)) + .isEqualTo(ApplicationStatus.SUCCEEDED); + } + + @Test + public void testFailedApplicationStatusFromJobStatus() { + assertThat(ApplicationStatus.fromJobStatus(JobStatus.FAILED)) + .isEqualTo(ApplicationStatus.FAILED); + } + + @Test + public void testCancelledApplicationStatusFromJobStatus() { + assertThat(ApplicationStatus.fromJobStatus(JobStatus.CANCELED)) + .isEqualTo(ApplicationStatus.CANCELED); + } + + @ParameterizedTest + @EnumSource( + value = JobStatus.class, + names = { + "INITIALIZING", + "CREATED", + "RUNNING", + "FAILING", + "CANCELLING", + "RESTARTING", + "SUSPENDED", + "RECONCILING" + }) + public void testUnknownApplicationStatusFromJobStatus(JobStatus jobStatus) { + assertThat(ApplicationStatus.fromJobStatus(jobStatus)).isEqualTo(ApplicationStatus.UNKNOWN); + } + + @Test + public void testUnknownApplicationStatusForMissingJobStatus() { + assertThat(ApplicationStatus.fromJobStatus(null)).isEqualTo(ApplicationStatus.UNKNOWN); } private static Iterable exitCodes(Iterable statuses) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java index 345aae7d1e9bb..7794de3333a49 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java @@ -25,16 +25,12 @@ import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; +import org.apache.flink.runtime.dispatcher.cleanup.CheckpointResourcesCleanupRunnerFactory; import org.apache.flink.runtime.heartbeat.HeartbeatServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.JobGraphWriter; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore; import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; -import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; -import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; -import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -50,11 +46,6 @@ import org.junit.rules.TemporaryFolder; import org.junit.rules.TestName; -import java.util.Collection; -import java.util.Collections; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ForkJoinPool; - /** Abstract test for the {@link Dispatcher} component. */ public class AbstractDispatcherTest extends TestLogger { @@ -106,12 +97,27 @@ public void setUp() throws Exception { haServices.setCheckpointRecoveryFactory(new StandaloneCheckpointRecoveryFactory()); haServices.setResourceManagerLeaderRetriever(new SettableLeaderRetrievalService()); haServices.setJobGraphStore(new StandaloneJobGraphStore()); + haServices.setJobResultStore(new EmbeddedJobResultStore()); configuration = new Configuration(); blobServer = new BlobServer(configuration, temporaryFolder.newFolder(), new VoidBlobStore()); } + protected TestingDispatcher.Builder createTestingDispatcherBuilder() { + return TestingDispatcher.builder() + .withRpcService(rpcService) + .withConfiguration(configuration) + .withHeartbeatServices(heartbeatServices) + .withHighAvailabilityServices(haServices) + .withJobGraphWriter(haServices.getJobGraphStore()) + .withJobResultStore(haServices.getJobResultStore()) + .withJobManagerRunnerFactory(JobMasterServiceLeadershipRunnerFactory.INSTANCE) + .withCleanupRunnerFactory(CheckpointResourcesCleanupRunnerFactory.INSTANCE) + .withFatalErrorHandler(testingFatalErrorHandlerResource.getFatalErrorHandler()) + .withBlobServer(blobServer); + } + @After public void tearDown() throws Exception { if (haServices != null) { @@ -125,93 +131,4 @@ public void tearDown() throws Exception { protected BlobServer getBlobServer() { return blobServer; } - - /** A convenient builder for the {@link TestingDispatcher}. */ - public class TestingDispatcherBuilder { - - private Collection initialJobGraphs = Collections.emptyList(); - - private final DispatcherBootstrapFactory dispatcherBootstrapFactory = - (dispatcher, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(); - - private HeartbeatServices heartbeatServices = AbstractDispatcherTest.this.heartbeatServices; - - private HighAvailabilityServices haServices = AbstractDispatcherTest.this.haServices; - - private JobManagerRunnerFactory jobManagerRunnerFactory = - JobMasterServiceLeadershipRunnerFactory.INSTANCE; - - private JobGraphWriter jobGraphWriter = NoOpJobGraphWriter.INSTANCE; - - private FatalErrorHandler fatalErrorHandler = - testingFatalErrorHandlerResource.getFatalErrorHandler(); - - private HistoryServerArchivist historyServerArchivist = VoidHistoryServerArchivist.INSTANCE; - - TestingDispatcherBuilder setHeartbeatServices(HeartbeatServices heartbeatServices) { - this.heartbeatServices = heartbeatServices; - return this; - } - - TestingDispatcherBuilder setHaServices(HighAvailabilityServices haServices) { - this.haServices = haServices; - return this; - } - - TestingDispatcherBuilder setInitialJobGraphs(Collection initialJobGraphs) { - this.initialJobGraphs = initialJobGraphs; - return this; - } - - TestingDispatcherBuilder setJobManagerRunnerFactory( - JobManagerRunnerFactory jobManagerRunnerFactory) { - this.jobManagerRunnerFactory = jobManagerRunnerFactory; - return this; - } - - TestingDispatcherBuilder setJobGraphWriter(JobGraphWriter jobGraphWriter) { - this.jobGraphWriter = jobGraphWriter; - return this; - } - - public TestingDispatcherBuilder setFatalErrorHandler(FatalErrorHandler fatalErrorHandler) { - this.fatalErrorHandler = fatalErrorHandler; - return this; - } - - public TestingDispatcherBuilder setHistoryServerArchivist( - HistoryServerArchivist historyServerArchivist) { - this.historyServerArchivist = historyServerArchivist; - return this; - } - - TestingDispatcher build() throws Exception { - TestingResourceManagerGateway resourceManagerGateway = - new TestingResourceManagerGateway(); - - final MemoryExecutionGraphInfoStore executionGraphInfoStore = - new MemoryExecutionGraphInfoStore(); - - return new TestingDispatcher( - rpcService, - DispatcherId.generate(), - initialJobGraphs, - dispatcherBootstrapFactory, - new DispatcherServices( - configuration, - haServices, - () -> CompletableFuture.completedFuture(resourceManagerGateway), - blobServer, - heartbeatServices, - executionGraphInfoStore, - fatalErrorHandler, - historyServerArchivist, - null, - new DispatcherOperationCaches(), - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), - jobGraphWriter, - jobManagerRunnerFactory, - ForkJoinPool.commonPool())); - } - } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java index 67271d6b2249b..6e6c9cb360634 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherFailoverITCase.java @@ -19,12 +19,11 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Deadline; import org.apache.flink.runtime.checkpoint.EmbeddedCompletedCheckpointStore; -import org.apache.flink.runtime.checkpoint.JobManagerTaskRestore; import org.apache.flink.runtime.checkpoint.PerJobCheckpointRecoveryFactory; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.execution.ExecutionState; -import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphBuilder; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -32,14 +31,19 @@ import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; import org.apache.flink.runtime.jobmaster.JobMasterGateway; import org.apache.flink.runtime.jobmaster.JobMasterId; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcEndpoint; import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.TestingJobGraphStore; import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.TimeUtils; +import org.hamcrest.CoreMatchers; +import org.hamcrest.collection.IsEmptyCollection; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -49,15 +53,16 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Objects; import java.util.Optional; import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; -import static org.junit.Assert.assertFalse; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertTrue; /** An integration test for various fail-over scenarios of the {@link Dispatcher} component. */ @@ -72,10 +77,11 @@ public void setUp() throws Exception { new PerJobCheckpointRecoveryFactory( (maxCheckpoints, previous, sharedStateRegistryFactory, ioExecutor) -> { if (previous != null) { - // First job attempt failed before cleaning up the checkpoint - // store. - assertFalse(previous.getShutdownStatus().isPresent()); - assertFalse(previous.getAllCheckpoints().isEmpty()); + // First job cleanup still succeeded for the + // CompletedCheckpointStore because the JobGraph cleanup happens + // after the JobManagerRunner closing + assertTrue(previous.getShutdownStatus().isPresent()); + assertTrue(previous.getAllCheckpoints().isEmpty()); return new EmbeddedCompletedCheckpointStore( maxCheckpoints, previous.getAllCheckpoints(), @@ -109,12 +115,17 @@ public void testRecoverFromCheckpointAfterJobGraphRemovalOfTerminatedJobFailed() final JobID jobId = jobGraph.getJobID(); // Construct job graph store. - final Error jobGraphRemovalError = new Error("Unable to remove job graph."); + final Error temporaryError = new Error("Unable to remove job graph."); + final AtomicReference temporaryErrorRef = + new AtomicReference<>(temporaryError); final TestingJobGraphStore jobGraphStore = TestingJobGraphStore.newBuilder() - .setRemoveJobGraphConsumer( + .setGlobalCleanupConsumer( graph -> { - throw jobGraphRemovalError; + if (temporaryErrorRef.get() != null) { + // only make the failure happen once + throw temporaryErrorRef.getAndSet(null); + } }) .build(); jobGraphStore.start(null); @@ -132,8 +143,7 @@ public void testRecoverFromCheckpointAfterJobGraphRemovalOfTerminatedJobFailed() throwable -> { final Optional maybeError = ExceptionUtils.findThrowable(throwable, Error.class); - if (maybeError.isPresent() - && jobGraphRemovalError.equals(maybeError.get())) { + if (maybeError.isPresent() && temporaryError.equals(maybeError.get())) { jobGraphRemovalErrorReceived.countDown(); } else { testingFatalErrorHandlerResource @@ -169,28 +179,33 @@ public void testRecoverFromCheckpointAfterJobGraphRemovalOfTerminatedJobFailed() // This will clear internal state of election service, so a new contender can register. leaderElectionService.stop(); + assertThat( + "The JobGraph is still stored in the JobGraphStore.", + haServices.getJobGraphStore().getJobIds(), + CoreMatchers.is(Collections.singleton(jobId))); + assertThat( + "The JobResultStore has this job marked as dirty.", + haServices.getJobResultStore().getDirtyResults().stream() + .map(JobResult::getJobId) + .collect(Collectors.toSet()), + CoreMatchers.is(Collections.singleton(jobId))); + // Run a second dispatcher, that restores our finished job. final Dispatcher secondDispatcher = createRecoveredDispatcher(null); toTerminate.add(secondDispatcher); - final DispatcherGateway secondDispatcherGateway = - secondDispatcher.getSelfGateway(DispatcherGateway.class); leaderElectionService.isLeader(UUID.randomUUID()); - // Now make sure that restored job started from checkpoint. - final JobMasterGateway secondJobMasterGateway = - connectToLeadingJobMaster(leaderElectionService).get(); - try (final JobMasterTester tester = - new JobMasterTester(rpcService, jobId, secondJobMasterGateway)) { - final CompletableFuture> descriptorsFuture = - tester.deployVertices(2); - awaitStatus(secondDispatcherGateway, jobId, JobStatus.RUNNING); - final Optional maybeRestore = - descriptorsFuture.get().stream() - .map(TaskDeploymentDescriptor::getTaskRestore) - .filter(Objects::nonNull) - .findAny(); - assertTrue("Job has recovered from checkpoint.", maybeRestore.isPresent()); - } + CommonTestUtils.waitUntilCondition( + () -> haServices.getJobResultStore().getDirtyResults().isEmpty(), + Deadline.fromNow(TimeUtils.toDuration(TIMEOUT))); + + assertThat( + "The JobGraph is not stored in the JobGraphStore.", + haServices.getJobGraphStore().getJobIds(), + IsEmptyCollection.empty()); + assertTrue( + "The JobResultStore has the job listed as clean.", + haServices.getJobResultStore().hasJobResultEntry(jobId)); } private JobGraph createJobGraph() { @@ -221,16 +236,16 @@ private TestingDispatcher createRecoveredDispatcher( @Nullable FatalErrorHandler fatalErrorHandler) throws Exception { final List jobGraphs = new ArrayList<>(); for (JobID jobId : haServices.getJobGraphStore().getJobIds()) { - jobGraphs.add(haServices.getJobGraphStore().recoverJobGraph(jobId)); + // there shouldn't be an overlap between dirty JobResults and recovered JobGraphs + if (!haServices.getJobResultStore().hasJobResultEntry(jobId)) { + jobGraphs.add(haServices.getJobGraphStore().recoverJobGraph(jobId)); + } } - haServices.setRunningJobsRegistry(new StandaloneRunningJobsRegistry()); final TestingDispatcher dispatcher = - new TestingDispatcherBuilder() - .setJobManagerRunnerFactory( - JobMasterServiceLeadershipRunnerFactory.INSTANCE) - .setJobGraphWriter(haServices.getJobGraphStore()) - .setInitialJobGraphs(jobGraphs) - .setFatalErrorHandler( + createTestingDispatcherBuilder() + .withRecoveredJobs(jobGraphs) + .withRecoveredDirtyJobs(haServices.getJobResultStore().getDirtyResults()) + .withFatalErrorHandler( fatalErrorHandler == null ? testingFatalErrorHandlerResource.getFatalErrorHandler() : fatalErrorHandler) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 80123ea3f1c7d..f62e7f5da5722 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -24,42 +24,43 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.core.testutils.OneShotLatch; -import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.blob.BlobServer; -import org.apache.flink.runtime.blob.BlobStore; -import org.apache.flink.runtime.blob.PermanentBlobKey; -import org.apache.flink.runtime.blob.TestingBlobStore; +import org.apache.flink.runtime.blob.BlobUtils; import org.apache.flink.runtime.blob.TestingBlobStoreBuilder; import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.client.JobSubmissionException; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.dispatcher.cleanup.TestingResourceCleanerFactory; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; -import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; -import org.apache.flink.runtime.jobmanager.JobGraphWriter; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; +import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway; +import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; -import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; -import org.apache.flink.runtime.testutils.TestingJobGraphStore; +import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; +import org.apache.flink.util.Reference; import org.apache.flink.util.TestLogger; -import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.function.ThrowingRunnable; +import org.hamcrest.core.IsInstanceOf; import org.junit.After; import org.junit.AfterClass; import org.junit.Before; @@ -71,23 +72,28 @@ import org.junit.rules.TemporaryFolder; import javax.annotation.Nonnull; +import javax.annotation.Nullable; -import java.io.File; import java.io.IOException; import java.util.ArrayDeque; import java.util.Arrays; import java.util.Collections; import java.util.Optional; import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.core.testutils.FlinkMatchers.containsMessage; +import static org.apache.flink.runtime.dispatcher.AbstractDispatcherTest.awaitStatus; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; /** Tests the resource cleanup by the {@link Dispatcher}. */ @@ -109,11 +115,7 @@ public class DispatcherResourceCleanupTest extends TestLogger { private JobGraph jobGraph; - private Configuration configuration; - - private SingleRunningJobsRegistry runningJobsRegistry; - - private TestingHighAvailabilityServices highAvailabilityServices; + private JobResultStore jobResultStore; private OneShotLatch clearedJobLatch; @@ -123,15 +125,8 @@ public class DispatcherResourceCleanupTest extends TestLogger { private BlobServer blobServer; - private PermanentBlobKey permanentBlobKey; - - private File blobFile; - - private CompletableFuture storedHABlobFuture; - private CompletableFuture deleteAllHABlobsFuture; - private CompletableFuture cleanupJobFuture; - private CompletableFuture cleanupJobHADataFuture; - private JobGraphWriter jobGraphWriter = NoOpJobGraphWriter.INSTANCE; + private CompletableFuture localCleanupFuture; + private CompletableFuture globalCleanupFuture; @BeforeClass public static void setupClass() { @@ -143,33 +138,17 @@ public void setup() throws Exception { jobGraph = JobGraphTestUtils.singleNoOpJobGraph(); jobId = jobGraph.getJobID(); - configuration = new Configuration(); - - highAvailabilityServices = new TestingHighAvailabilityServices(); clearedJobLatch = new OneShotLatch(); - runningJobsRegistry = new SingleRunningJobsRegistry(jobId, clearedJobLatch); - highAvailabilityServices.setRunningJobsRegistry(runningJobsRegistry); - cleanupJobHADataFuture = new CompletableFuture<>(); - highAvailabilityServices.setCleanupJobDataFuture(cleanupJobHADataFuture); - - storedHABlobFuture = new CompletableFuture<>(); - deleteAllHABlobsFuture = new CompletableFuture<>(); - - final TestingBlobStore testingBlobStore = - new TestingBlobStoreBuilder() - .setPutFunction( - (file, jobId, blobKey) -> storedHABlobFuture.complete(blobKey)) - .setDeleteAllFunction(deleteAllHABlobsFuture::complete) - .createTestingBlobStore(); + jobResultStore = new SingleJobResultStore(jobId, clearedJobLatch); - cleanupJobFuture = new CompletableFuture<>(); + globalCleanupFuture = new CompletableFuture<>(); + localCleanupFuture = new CompletableFuture<>(); blobServer = - new TestingBlobServer( - configuration, - temporaryFolder.newFolder(), - testingBlobStore, - cleanupJobFuture); + BlobUtils.createBlobServer( + new Configuration(), + Reference.owned(temporaryFolder.newFolder()), + new TestingBlobStoreBuilder().createTestingBlobStore()); } private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob() throws Exception { @@ -178,8 +157,8 @@ private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob() throws Exce private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob( int numBlockingJobManagerRunners) throws Exception { - final TestingJobManagerRunnerFactory testingJobManagerRunnerFactoryNG = - new TestingJobManagerRunnerFactory(numBlockingJobManagerRunners); + final TestingJobMasterServiceLeadershipRunnerFactory testingJobManagerRunnerFactoryNG = + new TestingJobMasterServiceLeadershipRunnerFactory(numBlockingJobManagerRunners); startDispatcher(testingJobManagerRunnerFactoryNG); submitJobAndWait(); @@ -187,32 +166,26 @@ private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob( } private void startDispatcher(JobManagerRunnerFactory jobManagerRunnerFactory) throws Exception { - TestingResourceManagerGateway resourceManagerGateway = new TestingResourceManagerGateway(); - final HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 1000L); - final MemoryExecutionGraphInfoStore archivedExecutionGraphStore = - new MemoryExecutionGraphInfoStore(); + final JobManagerRunnerRegistry jobManagerRunnerRegistry = new JobManagerRunnerRegistry(2); dispatcher = - new TestingDispatcher( - rpcService, - DispatcherId.generate(), - Collections.emptyList(), - (dispatcher, scheduledExecutor, errorHandler) -> - new NoOpDispatcherBootstrap(), - new DispatcherServices( - configuration, - highAvailabilityServices, - () -> CompletableFuture.completedFuture(resourceManagerGateway), - blobServer, - heartbeatServices, - archivedExecutionGraphStore, - testingFatalErrorHandlerResource.getFatalErrorHandler(), - VoidHistoryServerArchivist.INSTANCE, - null, - new DispatcherOperationCaches(), - UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), - jobGraphWriter, - jobManagerRunnerFactory, - ForkJoinPool.commonPool())); + TestingDispatcher.builder() + .withRpcService(rpcService) + .withJobResultStore(jobResultStore) + .withJobManagerRunnerRegistry(jobManagerRunnerRegistry) + .withBlobServer(blobServer) + .withFatalErrorHandler( + testingFatalErrorHandlerResource.getFatalErrorHandler()) + .withJobManagerRunnerFactory(jobManagerRunnerFactory) + .withResourceCleanerFactory( + new TestingResourceCleanerFactory() + // JobManagerRunnerRegistry needs to be added explicitly + // because cleaning it will trigger the closeAsync latch + // provided by TestingJobManagerRunner + .with(jobManagerRunnerRegistry) + .withGloballyCleanableResource( + globalCleanupFuture::complete) + .withLocallyCleanableResource(localCleanupFuture::complete)) + .build(); dispatcher.start(); @@ -224,6 +197,10 @@ public void teardown() throws Exception { if (dispatcher != null) { dispatcher.close(); } + + if (blobServer != null) { + blobServer.close(); + } } @AfterClass @@ -241,34 +218,11 @@ public void testBlobServerCleanupWhenJobFinished() throws Exception { // complete the job finishJob(jobManagerRunnerFactory.takeCreatedJobManagerRunner()); - assertThatHABlobsHaveBeenRemoved(); - } - - private void assertThatHABlobsHaveBeenRemoved() - throws InterruptedException, ExecutionException { - assertThat(cleanupJobFuture.get(), equalTo(jobId)); - - // verify that we also cleared the BlobStore - assertThat(deleteAllHABlobsFuture.get(), equalTo(jobId)); - - assertThat(blobFile.exists(), is(false)); + assertGlobalCleanupTriggered(jobId); } private CompletableFuture submitJob() { - try { - // upload a blob to the blob server - permanentBlobKey = blobServer.putPermanent(jobId, new byte[256]); - jobGraph.addUserJarBlobKey(permanentBlobKey); - blobFile = blobServer.getStorageLocation(jobId, permanentBlobKey); - - assertThat(blobFile.exists(), is(true)); - - // verify that we stored the blob also in the BlobStore - assertThat(storedHABlobFuture.join(), equalTo(permanentBlobKey)); - return dispatcherGateway.submitJob(jobGraph, timeout); - } catch (IOException ioe) { - return FutureUtils.completedExceptionally(ioe); - } + return dispatcherGateway.submitJob(jobGraph, timeout); } private void submitJobAndWait() { @@ -285,19 +239,7 @@ public void testBlobServerCleanupWhenJobNotFinished() throws Exception { jobManagerRunnerFactory.takeCreatedJobManagerRunner(); suspendJob(testingJobManagerRunner); - assertThat(cleanupJobFuture.get(), equalTo(jobId)); - - assertThat(blobFile.exists(), is(false)); - - // verify that we did not clear the BlobStore - try { - deleteAllHABlobsFuture.get(50L, TimeUnit.MILLISECONDS); - fail("We should not delete the HA blobs."); - } catch (TimeoutException ignored) { - // expected - } - - assertThat(deleteAllHABlobsFuture.isDone(), is(false)); + assertLocalCleanupTriggered(jobId); } /** Tests that the uploaded blobs are being cleaned up in case of a job submission failure. */ @@ -313,7 +255,7 @@ public void testBlobServerCleanupWhenJobSubmissionFails() throws Exception { assertThat(ee, FlinkMatchers.containsCause(JobSubmissionException.class)); } - assertThatHABlobsHaveBeenRemoved(); + assertGlobalCleanupTriggered(jobId); } @Test @@ -322,19 +264,7 @@ public void testBlobServerCleanupWhenClosingDispatcher() throws Exception { dispatcher.closeAsync().get(); - assertThat(cleanupJobFuture.get(), equalTo(jobId)); - - assertThat(blobFile.exists(), is(false)); - - // verify that we did not clear the BlobStore - try { - deleteAllHABlobsFuture.get(50L, TimeUnit.MILLISECONDS); - fail("We should not delete the HA blobs."); - } catch (TimeoutException ignored) { - // expected - } - - assertThat(deleteAllHABlobsFuture.isDone(), is(false)); + assertLocalCleanupTriggered(jobId); } @Test @@ -366,21 +296,23 @@ public void testHACleanupWhenJobFinishedWhileClosingDispatcher() throws Exceptio // check that no exceptions have been thrown dispatcherTerminationFuture.get(); - assertThat(cleanupJobFuture.get(), is(jobId)); - assertThat(deleteAllHABlobsFuture.get(), is(jobId)); + assertGlobalCleanupTriggered(jobId); } /** - * Tests that the {@link RunningJobsRegistry} entries are cleared after the job reached a + * Tests that the {@link JobResultStore} entries are marked as clean after the job reached a * terminal state. */ @Test - public void testRunningJobsRegistryCleanup() throws Exception { + public void testJobResultStoreCleanup() throws Exception { final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(); - runningJobsRegistry.setJobRunning(jobId); - assertThat(runningJobsRegistry.contains(jobId), is(true)); + final JobResult jobResult = + TestingJobResultStore.createJobResult(jobId, ApplicationStatus.UNKNOWN); + + jobResultStore.createDirtyResult(new JobResultEntry(jobResult)); + assertTrue(jobResultStore.hasJobResultEntry(jobId)); final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); @@ -394,7 +326,8 @@ public void testRunningJobsRegistryCleanup() throws Exception { // wait for the clearing clearedJobLatch.await(); - assertThat(runningJobsRegistry.contains(jobId), is(false)); + assertTrue(jobResultStore.hasJobResultEntry(jobId)); + assertTrue(jobResultStore.getDirtyResults().isEmpty()); } /** @@ -406,7 +339,6 @@ public void testJobSubmissionUnderSameJobId() throws Exception { final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(1); - runningJobsRegistry.setJobRunning(jobId); final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); suspendJob(testingJobManagerRunner); @@ -456,12 +388,12 @@ public void testDuplicateJobSubmissionDoesNotDeleteJobMetaData() throws Exceptio is(true)); } - assertThatHABlobsHaveNotBeenRemoved(); + assertThatNoCleanupWasTriggered(); } finally { finishJob(testingJobManagerRunnerFactoryNG.takeCreatedJobManagerRunner()); } - assertThatHABlobsHaveBeenRemoved(); + assertGlobalCleanupTriggered(jobId); } @Test @@ -470,8 +402,7 @@ public void testHaDataCleanupWhenJobFinished() throws Exception { TestingJobManagerRunner jobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); finishJob(jobManagerRunner); - JobID jobID = cleanupJobHADataFuture.get(2000, TimeUnit.MILLISECONDS); - assertThat(jobID, is(this.jobId)); + assertGlobalCleanupTriggered(jobId); } @Test @@ -480,13 +411,7 @@ public void testHaDataCleanupWhenJobNotFinished() throws Exception { TestingJobManagerRunner jobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); suspendJob(jobManagerRunner); - try { - cleanupJobHADataFuture.get(10L, TimeUnit.MILLISECONDS); - fail("We should not delete the HA data for job."); - } catch (TimeoutException ignored) { - // expected - } - assertThat(cleanupJobHADataFuture.isDone(), is(false)); + assertLocalCleanupTriggered(jobId); } private void finishJob(TestingJobManagerRunner takeCreatedJobManagerRunner) { @@ -507,10 +432,9 @@ private void terminateJobWithState( .build())); } - private void assertThatHABlobsHaveNotBeenRemoved() { - assertThat(cleanupJobFuture.isDone(), is(false)); - assertThat(deleteAllHABlobsFuture.isDone(), is(false)); - assertThat(blobFile.exists(), is(true)); + private void assertThatNoCleanupWasTriggered() { + assertThat(globalCleanupFuture.isDone(), is(false)); + assertThat(localCleanupFuture.isDone(), is(false)); } @Test @@ -545,27 +469,23 @@ public void testDispatcherTerminationWaitsForJobMasterTerminations() throws Exce dispatcherTerminationFuture.get(); } - private static final class SingleRunningJobsRegistry implements RunningJobsRegistry { + private static final class SingleJobResultStore implements JobResultStore { - @Nonnull private final JobID expectedJobId; + private final JobID expectedJobId; + @Nullable private JobResultEntry actualJobResultEntry; + private boolean isDirty = true; - @Nonnull private final OneShotLatch clearedJobLatch; + private final OneShotLatch clearedJobLatch; - private JobSchedulingStatus jobSchedulingStatus = JobSchedulingStatus.PENDING; - - private boolean containsJob = false; - - private SingleRunningJobsRegistry( - @Nonnull JobID expectedJobId, @Nonnull OneShotLatch clearedJobLatch) { + private SingleJobResultStore(JobID expectedJobId, OneShotLatch clearedJobLatch) { this.expectedJobId = expectedJobId; this.clearedJobLatch = clearedJobLatch; } @Override - public void setJobRunning(JobID jobID) { - checkJobId(jobID); - containsJob = true; - jobSchedulingStatus = JobSchedulingStatus.RUNNING; + public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException { + checkJobId(jobResultEntry.getJobId()); + this.actualJobResultEntry = jobResultEntry; } private void checkJobId(JobID jobID) { @@ -573,40 +493,79 @@ private void checkJobId(JobID jobID) { } @Override - public void setJobFinished(JobID jobID) { - checkJobId(jobID); - containsJob = true; - jobSchedulingStatus = JobSchedulingStatus.DONE; + public void markResultAsClean(JobID jobId) throws IOException { + checkJobId(jobId); + Preconditions.checkNotNull(actualJobResultEntry); + isDirty = false; + clearedJobLatch.trigger(); } @Override - public JobSchedulingStatus getJobSchedulingStatus(JobID jobID) { - checkJobId(jobID); - return jobSchedulingStatus; + public boolean hasDirtyJobResultEntry(JobID jobId) throws IOException { + if (actualJobResultEntry == null) { + return false; + } + + checkJobId(jobId); + return isDirty; } - public boolean contains(JobID jobId) { + @Override + public boolean hasCleanJobResultEntry(JobID jobId) throws IOException { + if (actualJobResultEntry == null) { + return false; + } + checkJobId(jobId); - return containsJob; + return !isDirty; } @Override - public void clearJob(JobID jobID) { - checkJobId(jobID); - containsJob = false; - clearedJobLatch.trigger(); + public Set getDirtyResults() throws IOException { + return actualJobResultEntry != null && isDirty + ? Collections.singleton(actualJobResultEntry.getJobResult()) + : Collections.emptySet(); } } @Test - public void testHABlobsAreNotRemovedIfHAJobGraphRemovalFails() throws Exception { - jobGraphWriter = - TestingJobGraphStore.newBuilder() - .setRemoveJobGraphConsumer( - ignored -> { - throw new Exception("Failed to Remove future"); + public void testHABlobsAreRemovedIfHAJobGraphRemovalSucceeds() throws Exception { + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = + startDispatcherAndSubmitJob(); + + ArchivedExecutionGraph executionGraph = + new ArchivedExecutionGraphBuilder() + .setJobID(jobId) + .setState(JobStatus.CANCELED) + .build(); + + final TestingJobManagerRunner testingJobManagerRunner = + jobManagerRunnerFactory.takeCreatedJobManagerRunner(); + testingJobManagerRunner.completeResultFuture(new ExecutionGraphInfo(executionGraph)); + + assertGlobalCleanupTriggered(jobId); + } + + private void assertLocalCleanupTriggered(JobID jobId) + throws ExecutionException, InterruptedException, TimeoutException { + assertThat(localCleanupFuture.get(100, TimeUnit.MILLISECONDS), equalTo(jobId)); + assertThat(globalCleanupFuture.isDone(), is(false)); + } + + private void assertGlobalCleanupTriggered(JobID jobId) + throws ExecutionException, InterruptedException, TimeoutException { + assertThat(localCleanupFuture.isDone(), is(false)); + assertThat(globalCleanupFuture.get(100, TimeUnit.MILLISECONDS), equalTo(jobId)); + } + + @Test + public void testFatalErrorIfJobCannotBeMarkedDirtyInJobResultStore() throws Exception { + jobResultStore = + TestingJobResultStore.builder() + .withCreateDirtyResultConsumer( + jobResult -> { + throw new IOException("Expected IOException."); }) - .withAutomaticStart() .build(); final TestingJobManagerRunnerFactory jobManagerRunnerFactory = @@ -615,64 +574,165 @@ public void testHABlobsAreNotRemovedIfHAJobGraphRemovalFails() throws Exception ArchivedExecutionGraph executionGraph = new ArchivedExecutionGraphBuilder() .setJobID(jobId) - .setState(JobStatus.CANCELED) + .setState(JobStatus.FINISHED) .build(); final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); testingJobManagerRunner.completeResultFuture(new ExecutionGraphInfo(executionGraph)); - assertThat(cleanupJobFuture.get(), equalTo(jobId)); - assertThat(deleteAllHABlobsFuture.isDone(), is(false)); + final CompletableFuture errorFuture = + this.testingFatalErrorHandlerResource.getFatalErrorHandler().getErrorFuture(); + assertThat( + errorFuture.get(100, TimeUnit.MILLISECONDS), + IsInstanceOf.instanceOf(FlinkException.class)); + testingFatalErrorHandlerResource.getFatalErrorHandler().clearError(); } @Test - public void testHABlobsAreRemovedIfHAJobGraphRemovalSucceeds() throws Exception { + public void testErrorHandlingIfJobCannotBeMarkedAsCleanInJobResultStore() throws Exception { + final CompletableFuture dirtyJobFuture = new CompletableFuture<>(); + jobResultStore = + TestingJobResultStore.builder() + .withCreateDirtyResultConsumer(dirtyJobFuture::complete) + .withMarkResultAsCleanConsumer( + jobId -> { + throw new IOException("Expected IOException."); + }) + .build(); + final TestingJobManagerRunnerFactory jobManagerRunnerFactory = startDispatcherAndSubmitJob(); ArchivedExecutionGraph executionGraph = new ArchivedExecutionGraphBuilder() .setJobID(jobId) - .setState(JobStatus.CANCELED) + .setState(JobStatus.FINISHED) .build(); final TestingJobManagerRunner testingJobManagerRunner = jobManagerRunnerFactory.takeCreatedJobManagerRunner(); testingJobManagerRunner.completeResultFuture(new ExecutionGraphInfo(executionGraph)); - assertThat(cleanupJobFuture.get(), equalTo(jobId)); - assertThat(deleteAllHABlobsFuture.get(), equalTo(jobId)); - } - - private static final class TestingBlobServer extends BlobServer { - - private final CompletableFuture cleanupJobFuture; - - /** - * Instantiates a new BLOB server and binds it to a free network port. - * - * @param config Configuration to be used to instantiate the BlobServer - * @param blobStore BlobStore to store blobs persistently - * @param cleanupJobFuture - * @throws IOException thrown if the BLOB server cannot bind to a free network port or if - * the (local or distributed) file storage cannot be created or is not usable - */ - public TestingBlobServer( - Configuration config, - File storageDirectory, - BlobStore blobStore, - CompletableFuture cleanupJobFuture) - throws IOException { - super(config, storageDirectory, blobStore); - this.cleanupJobFuture = cleanupJobFuture; + final CompletableFuture errorFuture = + this.testingFatalErrorHandlerResource.getFatalErrorHandler().getErrorFuture(); + try { + final Throwable unexpectedError = errorFuture.get(100, TimeUnit.MILLISECONDS); + fail( + "No error should have been reported but an " + + unexpectedError.getClass() + + " was handled."); + } catch (TimeoutException e) { + // expected + } + + assertThat(dirtyJobFuture.get().getJobId(), is(jobId)); + } + + /** Tests that a failing {@link JobManagerRunner} will be properly cleaned up. */ + @Test + public void testFailingJobManagerRunnerCleanup() throws Exception { + final FlinkException testException = new FlinkException("Test exception."); + final ArrayBlockingQueue> queue = new ArrayBlockingQueue<>(2); + + final BlockingJobManagerRunnerFactory blockingJobManagerRunnerFactory = + new BlockingJobManagerRunnerFactory( + () -> { + final Optional maybeException = queue.take(); + if (maybeException.isPresent()) { + throw maybeException.get(); + } + }); + + startDispatcher(blockingJobManagerRunnerFactory); + + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + + // submit and fail during job master runner construction + queue.offer(Optional.of(testException)); + try { + dispatcherGateway.submitJob(jobGraph, Time.minutes(1)).get(); + fail("A FlinkException is expected"); + } catch (Throwable expectedException) { + assertThat(expectedException, containsCause(FlinkException.class)); + assertThat(expectedException, containsMessage(testException.getMessage())); + // make sure we've cleaned up in correct order (including HA) + assertGlobalCleanupTriggered(jobId); + } + + // don't fail this time + queue.offer(Optional.empty()); + // submit job again + dispatcherGateway.submitJob(jobGraph, Time.minutes(1L)).get(); + blockingJobManagerRunnerFactory.setJobStatus(JobStatus.RUNNING); + + // Ensure job is running + awaitStatus(dispatcherGateway, jobId, JobStatus.RUNNING); + } + + private static final class BlockingJobManagerRunnerFactory + extends TestingJobMasterServiceLeadershipRunnerFactory { + + @Nonnull private final ThrowingRunnable jobManagerRunnerCreationLatch; + private TestingJobManagerRunner testingRunner; + + BlockingJobManagerRunnerFactory( + @Nonnull ThrowingRunnable jobManagerRunnerCreationLatch) { + super(0); + this.jobManagerRunnerCreationLatch = jobManagerRunnerCreationLatch; } @Override - public boolean cleanupJob(JobID jobId, boolean cleanupBlobStoreFiles) { - final boolean result = super.cleanupJob(jobId, cleanupBlobStoreFiles); - cleanupJobFuture.complete(jobId); - return result; + public TestingJobManagerRunner createJobManagerRunner( + JobGraph jobGraph, + Configuration configuration, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + HeartbeatServices heartbeatServices, + JobManagerSharedServices jobManagerSharedServices, + JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, + FatalErrorHandler fatalErrorHandler, + long initializationTimestamp) + throws Exception { + jobManagerRunnerCreationLatch.run(); + + this.testingRunner = + super.createJobManagerRunner( + jobGraph, + configuration, + rpcService, + highAvailabilityServices, + heartbeatServices, + jobManagerSharedServices, + jobManagerJobMetricGroupFactory, + fatalErrorHandler, + initializationTimestamp); + + TestingJobMasterGateway testingJobMasterGateway = + new TestingJobMasterGatewayBuilder() + .setRequestJobSupplier( + () -> + CompletableFuture.completedFuture( + new ExecutionGraphInfo( + ArchivedExecutionGraph + .createSparseArchivedExecutionGraph( + jobGraph.getJobID(), + jobGraph.getName(), + JobStatus.RUNNING, + null, + null, + 1337)))) + .build(); + testingRunner.completeJobMasterGatewayFuture(testingJobMasterGateway); + return testingRunner; + } + + public void setJobStatus(JobStatus newStatus) { + Preconditions.checkState( + testingRunner != null, + "JobManagerRunner must be created before this method is available"); + this.testingRunner.setJobStatus(newStatus); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index 4f9985f61f90a..5d3ffecb8509c 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -33,14 +33,15 @@ import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.dispatcher.cleanup.TestingCleanupRunnerFactory; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; import org.apache.flink.runtime.executiongraph.ErrorInfo; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; -import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphBuilder; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; @@ -58,7 +59,6 @@ import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; import org.apache.flink.runtime.jobmaster.factories.JobMasterServiceProcessFactory; import org.apache.flink.runtime.jobmaster.factories.TestingJobMasterServiceFactory; -import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGateway; import org.apache.flink.runtime.jobmaster.utils.TestingJobMasterGatewayBuilder; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; @@ -80,13 +80,13 @@ import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.TestingJobGraphStore; +import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; import org.apache.flink.util.TimeUtils; -import org.apache.flink.util.function.ThrowingRunnable; import org.assertj.core.api.Assertions; import org.hamcrest.Matchers; @@ -107,11 +107,9 @@ import java.nio.file.Paths; import java.time.Duration; import java.util.ArrayDeque; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.Optional; import java.util.Queue; import java.util.UUID; import java.util.concurrent.ArrayBlockingQueue; @@ -124,8 +122,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; -import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; -import static org.apache.flink.core.testutils.FlinkMatchers.containsMessage; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.instanceOf; @@ -144,7 +141,7 @@ public class DispatcherTest extends AbstractDispatcherTest { private static final String CLEANUP_JOB_GRAPH_RELEASE = "job-graph-release"; private static final String CLEANUP_JOB_MANAGER_RUNNER = "job-manager-runner"; private static final String CLEANUP_HA_SERVICES = "ha-services"; - private static final String CLEANUP_RUNNING_JOBS_REGISTRY = "running-jobs-registry"; + private static final String CLEANUP_JOB_RESULT_STORE = "job-result-store"; private JobGraph jobGraph; @@ -174,11 +171,12 @@ private TestingDispatcher createAndStartDispatcher( JobManagerRunnerFactory jobManagerRunnerFactory) throws Exception { final TestingDispatcher dispatcher = - new TestingDispatcherBuilder() - .setHaServices(haServices) - .setHeartbeatServices(heartbeatServices) - .setJobManagerRunnerFactory(jobManagerRunnerFactory) - .setJobGraphWriter(haServices.getJobGraphStore()) + createTestingDispatcherBuilder() + .withHighAvailabilityServices(haServices) + .withHeartbeatServices(heartbeatServices) + .withJobManagerRunnerFactory(jobManagerRunnerFactory) + .withJobGraphWriter(haServices.getJobGraphStore()) + .withJobResultStore(haServices.getJobResultStore()) .build(); dispatcher.start(); return dispatcher; @@ -216,7 +214,10 @@ public void testJobSubmission() throws Exception { @Test public void testDuplicateJobSubmissionWithGloballyTerminatedJobId() throws Exception { - haServices.getRunningJobsRegistry().setJobFinished(jobGraph.getJobID()); + final JobResult jobResult = + TestingJobResultStore.createJobResult( + jobGraph.getJobID(), ApplicationStatus.UNKNOWN); + haServices.getJobResultStore().createDirtyResult(new JobResultEntry(jobResult)); dispatcher = createAndStartDispatcher( heartbeatServices, @@ -238,11 +239,11 @@ public void testDuplicateJobSubmissionWithGloballyTerminatedJobId() throws Excep @Test public void testDuplicateJobSubmissionWithRunningJobId() throws Exception { dispatcher = - new TestingDispatcherBuilder() - .setJobManagerRunnerFactory( + createTestingDispatcherBuilder() + .withJobManagerRunnerFactory( new ExpectedJobIdJobManagerRunnerFactory( jobId, createdJobManagerRunnerLatch)) - .setInitialJobGraphs(Collections.singleton(jobGraph)) + .withRecoveredJobs(Collections.singleton(jobGraph)) .build(); dispatcher.start(); final DispatcherGateway dispatcherGateway = @@ -462,11 +463,11 @@ public void testNoHistoryServerArchiveCreatedForSuspendedJob() throws Exception final CompletableFuture jobTerminationFuture = new CompletableFuture<>(); dispatcher = - new TestingDispatcherBuilder() - .setJobManagerRunnerFactory( + createTestingDispatcherBuilder() + .withJobManagerRunnerFactory( new FinishingJobManagerRunnerFactory( jobTerminationFuture, () -> {})) - .setHistoryServerArchivist( + .withHistoryServerArchivist( executionGraphInfo -> { archiveAttemptFuture.complete(null); return CompletableFuture.completedFuture(null); @@ -498,8 +499,8 @@ public void testNoHistoryServerArchiveCreatedForSuspendedJob() throws Exception @Test public void testJobManagerRunnerInitializationFailureFailsJob() throws Exception { - final TestingJobManagerRunnerFactory testingJobManagerRunnerFactory = - new TestingJobManagerRunnerFactory(); + final TestingJobMasterServiceLeadershipRunnerFactory testingJobManagerRunnerFactory = + new TestingJobMasterServiceLeadershipRunnerFactory(); dispatcher = createAndStartDispatcher( @@ -519,7 +520,7 @@ public void testJobManagerRunnerInitializationFailureFailsJob() throws Exception testingJobManagerRunner.completeResultFuture( JobManagerRunnerResult.forInitializationFailure( new ExecutionGraphInfo( - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, jobGraph.getName(), JobStatus.FAILED, @@ -660,13 +661,12 @@ public void testFatalErrorIfRecoveredJobsCannotBeStarted() throws Exception { final FlinkException testException = new FlinkException("Test exception"); jobMasterLeaderElectionService.isLeader(UUID.randomUUID()); - final TestingJobManagerRunnerFactory jobManagerRunnerFactory = - new TestingJobManagerRunnerFactory(); + final TestingJobMasterServiceLeadershipRunnerFactory jobManagerRunnerFactory = + new TestingJobMasterServiceLeadershipRunnerFactory(); dispatcher = - new TestingDispatcherBuilder() - .setJobManagerRunnerFactory(jobManagerRunnerFactory) - .setInitialJobGraphs( - Collections.singleton(JobGraphTestUtils.emptyJobGraph())) + createTestingDispatcherBuilder() + .withJobManagerRunnerFactory(jobManagerRunnerFactory) + .withRecoveredJobs(Collections.singleton(JobGraphTestUtils.emptyJobGraph())) .build(); dispatcher.start(); @@ -681,7 +681,7 @@ public void testFatalErrorIfRecoveredJobsCannotBeStarted() throws Exception { testingJobManagerRunner.completeResultFuture( JobManagerRunnerResult.forInitializationFailure( new ExecutionGraphInfo( - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, jobGraph.getName(), JobStatus.FAILED, @@ -703,82 +703,61 @@ public void testFatalErrorIfRecoveredJobsCannotBeStarted() throws Exception { fatalErrorHandler.clearError(); } - /** Tests that a failing {@link JobManagerRunner} will be properly cleaned up. */ - @Test - public void testFailingJobManagerRunnerCleanup() throws Exception { - final FlinkException testException = new FlinkException("Test exception."); - final ArrayBlockingQueue> queue = new ArrayBlockingQueue<>(2); - - final BlockingJobManagerRunnerFactory blockingJobManagerRunnerFactory = - new BlockingJobManagerRunnerFactory( - () -> { - final Optional maybeException = queue.take(); - if (maybeException.isPresent()) { - throw maybeException.get(); - } - }); - - final BlockingQueue cleanUpEvents = new LinkedBlockingQueue<>(); - - // Track cleanup - ha-services - final CompletableFuture cleanupJobData = new CompletableFuture<>(); - haServices.setCleanupJobDataFuture(cleanupJobData); - cleanupJobData.thenAccept(jobId -> cleanUpEvents.add(CLEANUP_HA_SERVICES)); - - // Track cleanup - job-graph - final TestingJobGraphStore jobGraphStore = - TestingJobGraphStore.newBuilder() - .setReleaseJobGraphConsumer( - jobId -> cleanUpEvents.add(CLEANUP_JOB_GRAPH_RELEASE)) - .setRemoveJobGraphConsumer( - jobId -> cleanUpEvents.add(CLEANUP_JOB_GRAPH_REMOVE)) + @Test(expected = IllegalArgumentException.class) + public void testThatDirtilyFinishedJobsNotBeingRetriggered() throws Exception { + final JobGraph jobGraph = JobGraphTestUtils.emptyJobGraph(); + final JobResult jobResult = + TestingJobResultStore.createSuccessfulJobResult(jobGraph.getJobID()); + dispatcher = + createTestingDispatcherBuilder() + .withRecoveredJobs(Collections.singleton(jobGraph)) + .withRecoveredDirtyJobs(Collections.singleton(jobResult)) .build(); - jobGraphStore.start(null); - haServices.setJobGraphStore(jobGraphStore); - - // Track cleanup - running jobs registry - haServices.setRunningJobsRegistry( - new StandaloneRunningJobsRegistry() { + } - @Override - public void clearJob(JobID jobID) { - super.clearJob(jobID); - cleanUpEvents.add(CLEANUP_RUNNING_JOBS_REGISTRY); - } - }); + @Test + public void testJobCleanupWithoutRecoveredJobGraph() throws Exception { + final JobID jobIdOfRecoveredDirtyJobs = new JobID(); + final TestingJobMasterServiceLeadershipRunnerFactory jobManagerRunnerFactory = + new TestingJobMasterServiceLeadershipRunnerFactory(); + final TestingCleanupRunnerFactory cleanupRunnerFactory = new TestingCleanupRunnerFactory(); + final OneShotLatch dispatcherBootstrapLatch = new OneShotLatch(); dispatcher = - createAndStartDispatcher( - heartbeatServices, haServices, blockingJobManagerRunnerFactory); + createTestingDispatcherBuilder() + .withJobManagerRunnerFactory(jobManagerRunnerFactory) + .withCleanupRunnerFactory(cleanupRunnerFactory) + .withRecoveredDirtyJobs( + Collections.singleton( + new JobResult.Builder() + .jobId(jobIdOfRecoveredDirtyJobs) + .applicationStatus(ApplicationStatus.SUCCEEDED) + .netRuntime(1) + .build())) + .withDispatcherBootstrapFactory( + (ignoredDispatcherGateway, + ignoredScheduledExecutor, + ignoredFatalErrorHandler) -> { + dispatcherBootstrapLatch.trigger(); + return new NoOpDispatcherBootstrap(); + }) + .build(); - final DispatcherGateway dispatcherGateway = - dispatcher.getSelfGateway(DispatcherGateway.class); + dispatcher.start(); - // submit and fail during job master runner construction - queue.offer(Optional.of(testException)); - try { - dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - } catch (Throwable expectedException) { - assertThat(expectedException, containsCause(FlinkException.class)); - assertThat(expectedException, containsMessage(testException.getMessage())); - // make sure we've cleaned up in correct order (including HA) - assertThat( - new ArrayList<>(cleanUpEvents), - equalTo( - Arrays.asList( - CLEANUP_JOB_GRAPH_REMOVE, - CLEANUP_RUNNING_JOBS_REGISTRY, - CLEANUP_HA_SERVICES))); - } + dispatcherBootstrapLatch.await(); - // don't fail this time - queue.offer(Optional.empty()); - // submit job again - dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - blockingJobManagerRunnerFactory.setJobStatus(JobStatus.RUNNING); + final TestingJobManagerRunner cleanupRunner = + cleanupRunnerFactory.takeCreatedJobManagerRunner(); + assertThat( + "The CleanupJobManagerRunner has the wrong job ID attached.", + cleanupRunner.getJobID(), + is(jobIdOfRecoveredDirtyJobs)); - // Ensure job is running - awaitStatus(dispatcherGateway, jobId, JobStatus.RUNNING); + assertThat( + "No JobMaster should have been started.", + jobManagerRunnerFactory.getQueueSize(), + is(0)); } @Test @@ -789,7 +768,7 @@ public void testPersistedJobGraphWhenDispatcherIsShutDown() throws Exception { haServices.setJobGraphStore(submittedJobGraphStore); dispatcher = - new TestingDispatcherBuilder().setJobGraphWriter(submittedJobGraphStore).build(); + createTestingDispatcherBuilder().withJobGraphWriter(submittedJobGraphStore).build(); dispatcher.start(); @@ -893,15 +872,15 @@ public void testOnRemovedJobGraphDoesNotCleanUpHAFiles() throws Exception { final TestingJobGraphStore testingJobGraphStore = TestingJobGraphStore.newBuilder() - .setRemoveJobGraphConsumer(removeJobGraphFuture::complete) - .setReleaseJobGraphConsumer(releaseJobGraphFuture::complete) + .setGlobalCleanupConsumer(removeJobGraphFuture::complete) + .setLocalCleanupConsumer(releaseJobGraphFuture::complete) .build(); testingJobGraphStore.start(null); dispatcher = - new TestingDispatcherBuilder() - .setInitialJobGraphs(Collections.singleton(jobGraph)) - .setJobGraphWriter(testingJobGraphStore) + createTestingDispatcherBuilder() + .withRecoveredJobs(Collections.singleton(jobGraph)) + .withJobGraphWriter(testingJobGraphStore) .build(); dispatcher.start(); @@ -1079,8 +1058,8 @@ public void testOnlyRecoveredJobsAreRetainedInTheBlobServer() throws Exception { final PermanentBlobKey blobKey2 = blobServer.putPermanent(jobId2, fileContent); dispatcher = - new TestingDispatcherBuilder() - .setInitialJobGraphs(Collections.singleton(new JobGraph(jobId1, "foobar"))) + createTestingDispatcherBuilder() + .withRecoveredJobs(Collections.singleton(new JobGraph(jobId1, "foobar"))) .build(); Assertions.assertThat(blobServer.getFile(jobId1, blobKey1)).hasBinaryContent(fileContent); @@ -1125,30 +1104,33 @@ private void testJobDataAreCleanedUpInCorrectOrder(JobStatus jobStatus) throws E // Track cleanup - ha-services final CompletableFuture cleanupJobData = new CompletableFuture<>(); - haServices.setCleanupJobDataFuture(cleanupJobData); + haServices.setGlobalCleanupFuture(cleanupJobData); cleanupJobData.thenAccept(jobId -> cleanUpEvents.add(CLEANUP_HA_SERVICES)); // Track cleanup - job-graph final TestingJobGraphStore jobGraphStore = TestingJobGraphStore.newBuilder() - .setReleaseJobGraphConsumer( + .setLocalCleanupConsumer( jobId -> cleanUpEvents.add(CLEANUP_JOB_GRAPH_RELEASE)) - .setRemoveJobGraphConsumer( + .setGlobalCleanupConsumer( jobId -> cleanUpEvents.add(CLEANUP_JOB_GRAPH_REMOVE)) .build(); jobGraphStore.start(null); haServices.setJobGraphStore(jobGraphStore); - // Track cleanup - running jobs registry - haServices.setRunningJobsRegistry( - new StandaloneRunningJobsRegistry() { - - @Override - public void clearJob(JobID jobID) { - super.clearJob(jobID); - cleanUpEvents.add(CLEANUP_RUNNING_JOBS_REGISTRY); - } - }); + // Track cleanup - job result store + haServices.setJobResultStore( + TestingJobResultStore.builder() + .withMarkResultAsCleanConsumer( + jobID -> + assertThat( + "All cleanup tasks should have been finished before marking the job as clean.", + cleanUpEvents, + containsInAnyOrder( + CLEANUP_HA_SERVICES, + CLEANUP_JOB_GRAPH_REMOVE, + CLEANUP_JOB_MANAGER_RUNNER))) + .build()); final CompletableFuture resultFuture = new CompletableFuture<>(); dispatcher = @@ -1164,19 +1146,14 @@ public void clearJob(JobID jobID) { resultFuture.complete( JobManagerRunnerResult.forSuccess( new ExecutionGraphInfo( - new ArchivedExecutionGraphBuilder().setState(jobStatus).build()))); + new ArchivedExecutionGraphBuilder() + .setState(jobStatus) + .setFailureCause( + new ErrorInfo(new RuntimeException("expected"), 1L)) + .build()))); // Wait for job to terminate. dispatcher.getJobTerminationFuture(jobId, TIMEOUT).get(); - - assertThat( - new ArrayList<>(cleanUpEvents), - equalTo( - Arrays.asList( - CLEANUP_JOB_GRAPH_REMOVE, - CLEANUP_JOB_MANAGER_RUNNER, - CLEANUP_RUNNING_JOBS_REGISTRY, - CLEANUP_HA_SERVICES))); } private static class JobManagerRunnerWithBlockingJobMasterFactory @@ -1233,7 +1210,7 @@ public JobManagerRunner createJobManagerRunner( })), highAvailabilityServices.getJobManagerLeaderElectionService( jobGraph.getJobID()), - highAvailabilityServices.getRunningJobsRegistry(), + highAvailabilityServices.getJobResultStore(), jobManagerServices .getLibraryCacheManager() .registerClassLoaderLease(jobGraph.getJobID()), @@ -1285,7 +1262,7 @@ public JobManagerRunner createJobManagerRunner( new TestingJobMasterServiceFactory()), highAvailabilityServices.getJobManagerLeaderElectionService( jobGraph.getJobID()), - highAvailabilityServices.getRunningJobsRegistry(), + highAvailabilityServices.getJobResultStore(), jobManagerServices .getLibraryCacheManager() .registerClassLoaderLease(jobGraph.getJobID()), @@ -1308,13 +1285,13 @@ public BlockingTerminationJobManagerService( CompletableFuture future, JobMasterServiceProcessFactory jobMasterServiceProcessFactory, LeaderElectionService leaderElectionService, - RunningJobsRegistry runningJobsRegistry, + JobResultStore jobResultStore, LibraryCacheManager.ClassLoaderLease classLoaderLease, FatalErrorHandler fatalErrorHandler) { super( jobMasterServiceProcessFactory, leaderElectionService, - runningJobsRegistry, + jobResultStore, classLoaderLease, fatalErrorHandler); this.future = future; @@ -1330,70 +1307,6 @@ public CompletableFuture closeAsync() { } } - private static final class BlockingJobManagerRunnerFactory - extends TestingJobManagerRunnerFactory { - - @Nonnull private final ThrowingRunnable jobManagerRunnerCreationLatch; - private TestingJobManagerRunner testingRunner; - - BlockingJobManagerRunnerFactory( - @Nonnull ThrowingRunnable jobManagerRunnerCreationLatch) { - this.jobManagerRunnerCreationLatch = jobManagerRunnerCreationLatch; - } - - @Override - public TestingJobManagerRunner createJobManagerRunner( - JobGraph jobGraph, - Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - HeartbeatServices heartbeatServices, - JobManagerSharedServices jobManagerSharedServices, - JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, - FatalErrorHandler fatalErrorHandler, - long initializationTimestamp) - throws Exception { - jobManagerRunnerCreationLatch.run(); - - this.testingRunner = - super.createJobManagerRunner( - jobGraph, - configuration, - rpcService, - highAvailabilityServices, - heartbeatServices, - jobManagerSharedServices, - jobManagerJobMetricGroupFactory, - fatalErrorHandler, - initializationTimestamp); - - TestingJobMasterGateway testingJobMasterGateway = - new TestingJobMasterGatewayBuilder() - .setRequestJobSupplier( - () -> - CompletableFuture.completedFuture( - new ExecutionGraphInfo( - ArchivedExecutionGraph - .createFromInitializingJob( - jobGraph.getJobID(), - jobGraph.getName(), - JobStatus.RUNNING, - null, - null, - 1337)))) - .build(); - testingRunner.completeJobMasterGatewayFuture(testingJobMasterGateway); - return testingRunner; - } - - public void setJobStatus(JobStatus newStatus) { - Preconditions.checkState( - testingRunner != null, - "JobManagerRunner must be created before this method is available"); - this.testingRunner.setJobStatus(newStatus); - } - } - private static final class InitializationTimestampCapturingJobManagerRunnerFactory implements JobManagerRunnerFactory { private final BlockingQueue initializationTimestampQueue; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobManagerRunnerRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobManagerRunnerRegistryTest.java new file mode 100644 index 0000000000000..314b038627de8 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobManagerRunnerRegistryTest.java @@ -0,0 +1,265 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobmaster.JobManagerRunner; +import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.concurrent.Executors; +import org.apache.flink.util.function.BiFunctionWithException; + +import org.assertj.core.api.ThrowingConsumer; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.NoSuchElementException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * {@code JobManagerRunnerRegistryTest} tests the functionality of {@link JobManagerRunnerRegistry}. + */ +public class JobManagerRunnerRegistryTest { + + private JobManagerRunnerRegistry testInstance; + + @BeforeEach + public void setup() { + testInstance = new JobManagerRunnerRegistry(4); + } + + @Test + public void testIsRegistered() { + final JobID jobId = new JobID(); + testInstance.register(TestingJobManagerRunner.newBuilder().setJobId(jobId).build()); + assertThat(testInstance.isRegistered(jobId)).isTrue(); + } + + @Test + public void testIsNotRegistered() { + assertThat(testInstance.isRegistered(new JobID())).isFalse(); + } + + @Test + public void testRegister() { + final JobID jobId = new JobID(); + testInstance.register(TestingJobManagerRunner.newBuilder().setJobId(jobId).build()); + assertThat(testInstance.isRegistered(jobId)).isTrue(); + } + + @Test + public void testRegisteringTwiceCausesFailure() { + final JobID jobId = new JobID(); + testInstance.register(TestingJobManagerRunner.newBuilder().setJobId(jobId).build()); + assertThat(testInstance.isRegistered(jobId)).isTrue(); + + assertThatThrownBy( + () -> + testInstance.register( + TestingJobManagerRunner.newBuilder() + .setJobId(jobId) + .build())) + .isInstanceOf(IllegalArgumentException.class); + } + + @Test + public void testGet() { + final JobID jobId = new JobID(); + final JobManagerRunner jobManagerRunner = + TestingJobManagerRunner.newBuilder().setJobId(jobId).build(); + testInstance.register(jobManagerRunner); + + assertThat(testInstance.get(jobId)).isEqualTo(jobManagerRunner); + } + + @Test + public void testGetOnNonExistingJobManagerRunner() { + assertThatThrownBy(() -> testInstance.get(new JobID())) + .isInstanceOf(NoSuchElementException.class); + } + + @Test + public void size() { + assertThat(testInstance.size()).isEqualTo(0); + testInstance.register(TestingJobManagerRunner.newBuilder().build()); + assertThat(testInstance.size()).isEqualTo(1); + testInstance.register(TestingJobManagerRunner.newBuilder().build()); + assertThat(testInstance.size()).isEqualTo(2); + } + + @Test + public void testGetRunningJobIds() { + assertThat(testInstance.getRunningJobIds()).isEmpty(); + + final JobID jobId0 = new JobID(); + final JobID jobId1 = new JobID(); + testInstance.register(TestingJobManagerRunner.newBuilder().setJobId(jobId0).build()); + testInstance.register(TestingJobManagerRunner.newBuilder().setJobId(jobId1).build()); + + assertThat(testInstance.getRunningJobIds()).containsExactlyInAnyOrder(jobId0, jobId1); + } + + @Test + public void testGetJobManagerRunners() { + assertThat(testInstance.getJobManagerRunners()).isEmpty(); + + final JobManagerRunner jobManagerRunner0 = TestingJobManagerRunner.newBuilder().build(); + final JobManagerRunner jobManagerRunner1 = TestingJobManagerRunner.newBuilder().build(); + testInstance.register(jobManagerRunner0); + testInstance.register(jobManagerRunner1); + + assertThat(testInstance.getJobManagerRunners()) + .containsExactlyInAnyOrder(jobManagerRunner0, jobManagerRunner1); + } + + @Test + public void testSuccessfulGlobalCleanup() throws Throwable { + testSuccessfulSynchronousCleanup(testInstance::globalCleanup); + } + + @Test + public void testSuccessfulLocalCleanup() throws Throwable { + testSuccessfulSynchronousCleanup(testInstance::localCleanup); + } + + private void testSuccessfulSynchronousCleanup(ThrowingConsumer callback) + throws Throwable { + final TestingJobManagerRunner jobManagerRunner = registerTestingJobManagerRunner(); + + callback.acceptThrows(jobManagerRunner.getJobID()); + assertThat(testInstance.isRegistered(jobManagerRunner.getJobID())).isFalse(); + assertThat(jobManagerRunner.getTerminationFuture()).isCompleted(); + } + + @Test + public void testFailingGlobalCleanup() { + testFailingSynchronousCleanup(testInstance::globalCleanup); + } + + @Test + public void testFailingLocalCleanup() { + testFailingSynchronousCleanup(testInstance::localCleanup); + } + + private void testFailingSynchronousCleanup(ThrowingConsumer callback) { + final TestingJobManagerRunner jobManagerRunner = registerTestingJobManagerRunner(); + + assertThat(testInstance.isRegistered(jobManagerRunner.getJobID())).isTrue(); + assertThat(jobManagerRunner.getTerminationFuture()).isNotDone(); + + final RuntimeException expectedException = new RuntimeException("Expected exception"); + jobManagerRunner.completeTerminationFutureExceptionally(expectedException); + + assertThatThrownBy(() -> callback.acceptThrows(jobManagerRunner.getJobID())) + .isInstanceOf(FlinkException.class) + .hasCause(expectedException); + assertThat(testInstance.isRegistered(jobManagerRunner.getJobID())).isFalse(); + } + + @Test + public void testSuccessfulGlobalCleanupAsync() throws Exception { + testSuccessfulCleanupAsync(testInstance::globalCleanupAsync); + } + + @Test + public void testSuccessfulLocalCleanupAsync() throws Exception { + testSuccessfulCleanupAsync(testInstance::localCleanupAsync); + } + + private void testSuccessfulCleanupAsync( + BiFunctionWithException, Exception> callback) + throws Exception { + final TestingJobManagerRunner jobManagerRunner = registerTestingJobManagerRunner(); + + final CompletableFuture cleanupResult = + callback.apply(jobManagerRunner.getJobID(), Executors.directExecutor()); + assertThat(testInstance.isRegistered(jobManagerRunner.getJobID())).isFalse(); + assertThat(cleanupResult).isCompleted(); + } + + @Test + public void testFailingGlobalCleanupAsync() throws Exception { + testFailingCleanupAsync(testInstance::globalCleanupAsync); + } + + @Test + public void testFailingLocalCleanupAsync() throws Exception { + testFailingCleanupAsync(testInstance::localCleanupAsync); + } + + private void testFailingCleanupAsync( + BiFunctionWithException, Exception> callback) + throws Exception { + final TestingJobManagerRunner jobManagerRunner = registerTestingJobManagerRunner(); + + assertThat(testInstance.isRegistered(jobManagerRunner.getJobID())).isTrue(); + assertThat(jobManagerRunner.getTerminationFuture()).isNotDone(); + + final RuntimeException expectedException = new RuntimeException("Expected exception"); + jobManagerRunner.completeTerminationFutureExceptionally(expectedException); + + final CompletableFuture cleanupResult = + callback.apply(jobManagerRunner.getJobID(), Executors.directExecutor()); + assertThat(testInstance.isRegistered(jobManagerRunner.getJobID())).isFalse(); + assertThat(cleanupResult) + .isCompletedExceptionally() + .failsWithin(Duration.ZERO) + .withThrowableOfType(ExecutionException.class) + .withCause(expectedException); + } + + private TestingJobManagerRunner registerTestingJobManagerRunner() { + final TestingJobManagerRunner jobManagerRunner = + TestingJobManagerRunner.newBuilder().build(); + testInstance.register(jobManagerRunner); + + assertThat(testInstance.isRegistered(jobManagerRunner.getJobID())).isTrue(); + assertThat(jobManagerRunner.getTerminationFuture()).isNotDone(); + + return jobManagerRunner; + } + + @Test + public void testGlobalCleanupAsyncOnUnknownJobId() { + assertThat(testInstance.globalCleanupAsync(new JobID(), Executors.directExecutor())) + .isCompleted(); + } + + @Test + public void testLocalCleanupAsyncOnUnknownJobId() { + assertThat(testInstance.localCleanupAsync(new JobID(), Executors.directExecutor())) + .isCompleted(); + } + + @Test + public void testGlobalCleanupOnUnknownJobId() throws Exception { + testInstance.globalCleanup(new JobID()); + } + + @Test + public void testLocalCleanupOnUnknownJobId() throws Exception { + testInstance.localCleanup(new JobID()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java index d76e4bbaa2dbc..04f559f5d9a3a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/MiniDispatcherTest.java @@ -18,12 +18,14 @@ package org.apache.flink.runtime.dispatcher; +import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.dispatcher.cleanup.TestingCleanupRunnerFactory; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; @@ -38,6 +40,7 @@ import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource; import org.apache.flink.util.TestLogger; @@ -49,7 +52,7 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.IOException; import java.util.concurrent.CompletableFuture; @@ -92,7 +95,8 @@ public class MiniDispatcherTest extends TestLogger { private TestingHighAvailabilityServices highAvailabilityServices; - private TestingJobManagerRunnerFactory testingJobManagerRunnerFactory; + private TestingJobMasterServiceLeadershipRunnerFactory testingJobManagerRunnerFactory; + private TestingCleanupRunnerFactory testingCleanupRunnerFactory; @BeforeClass public static void setupClass() throws IOException { @@ -116,7 +120,8 @@ public static void setupClass() throws IOException { public void setup() throws Exception { highAvailabilityServices = new TestingHighAvailabilityServicesBuilder().build(); - testingJobManagerRunnerFactory = new TestingJobManagerRunnerFactory(); + testingJobManagerRunnerFactory = new TestingJobMasterServiceLeadershipRunnerFactory(); + testingCleanupRunnerFactory = new TestingCleanupRunnerFactory(); } @AfterClass @@ -149,6 +154,27 @@ public void testSingleJobRecovery() throws Exception { } } + /** Tests that the {@link MiniDispatcher} recovers the single job with which it was started. */ + @Test + public void testDirtyJobResultCleanup() throws Exception { + final JobID jobId = new JobID(); + final MiniDispatcher miniDispatcher = + createMiniDispatcher( + ClusterEntrypoint.ExecutionMode.DETACHED, + null, + TestingJobResultStore.createSuccessfulJobResult(jobId)); + + miniDispatcher.start(); + + try { + final TestingJobManagerRunner testingCleanupRunner = + testingCleanupRunnerFactory.takeCreatedJobManagerRunner(); + assertThat(testingCleanupRunner.getJobID(), is(jobId)); + } finally { + RpcUtils.terminateRpcEndpoint(miniDispatcher, timeout); + } + } + /** * Tests that in detached mode, the {@link MiniDispatcher} will complete the future that signals * job termination. @@ -244,9 +270,16 @@ public void testShutdownIfJobCancelledInNormalMode() throws Exception { // Utilities // -------------------------------------------------------- - @Nonnull private MiniDispatcher createMiniDispatcher(ClusterEntrypoint.ExecutionMode executionMode) throws Exception { + return createMiniDispatcher(executionMode, jobGraph, null); + } + + private MiniDispatcher createMiniDispatcher( + ClusterEntrypoint.ExecutionMode executionMode, + @Nullable JobGraph recoveredJobGraph, + @Nullable JobResult recoveredDirtyJob) + throws Exception { return new MiniDispatcher( rpcService, DispatcherId.generate(), @@ -263,9 +296,12 @@ private MiniDispatcher createMiniDispatcher(ClusterEntrypoint.ExecutionMode exec new DispatcherOperationCaches(), UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(), highAvailabilityServices.getJobGraphStore(), + highAvailabilityServices.getJobResultStore(), testingJobManagerRunnerFactory, + testingCleanupRunnerFactory, ForkJoinPool.commonPool()), - jobGraph, + recoveredJobGraph, + recoveredDirtyJob, (dispatcher, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(), executionMode); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/NoOpJobGraphWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/NoOpJobGraphWriter.java index 7a2d156e1945c..bc17d94c145be 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/NoOpJobGraphWriter.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/NoOpJobGraphWriter.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.dispatcher; -import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmanager.JobGraphWriter; @@ -28,10 +27,4 @@ public enum NoOpJobGraphWriter implements JobGraphWriter { @Override public void putJobGraph(JobGraph jobGraph) throws Exception {} - - @Override - public void removeJobGraph(JobID jobId) throws Exception {} - - @Override - public void releaseJobGraph(JobID jobId) throws Exception {} } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java index 918d4f3cce0ba..0b3587e3cbaf1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java @@ -20,14 +20,41 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.dispatcher.cleanup.CleanupRunnerFactory; +import org.apache.flink.runtime.dispatcher.cleanup.DispatcherResourceCleanerFactory; +import org.apache.flink.runtime.dispatcher.cleanup.ResourceCleanerFactory; +import org.apache.flink.runtime.dispatcher.cleanup.TestingCleanupRunnerFactory; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.JobGraphWriter; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; +import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; +import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.runtime.util.TestingFatalErrorHandler; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; +import org.apache.flink.util.Preconditions; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.util.Collection; +import java.util.Collections; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; import java.util.function.Function; /** {@link Dispatcher} implementation used for testing purposes. */ @@ -39,6 +66,7 @@ class TestingDispatcher extends Dispatcher { RpcService rpcService, DispatcherId fencingToken, Collection recoveredJobs, + Collection recoveredDirtyJobResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, DispatcherServices dispatcherServices) throws Exception { @@ -46,12 +74,66 @@ class TestingDispatcher extends Dispatcher { rpcService, fencingToken, recoveredJobs, + recoveredDirtyJobResults, dispatcherBootstrapFactory, dispatcherServices); this.startFuture = new CompletableFuture<>(); } + private TestingDispatcher( + RpcService rpcService, + DispatcherId fencingToken, + Collection recoveredJobs, + Collection recoveredDirtyJobs, + Configuration configuration, + HighAvailabilityServices highAvailabilityServices, + GatewayRetriever resourceManagerGatewayRetriever, + HeartbeatServices heartbeatServices, + BlobServer blobServer, + FatalErrorHandler fatalErrorHandler, + JobGraphWriter jobGraphWriter, + JobResultStore jobResultStore, + JobManagerMetricGroup jobManagerMetricGroup, + @Nullable String metricServiceQueryAddress, + Executor ioExecutor, + HistoryServerArchivist historyServerArchivist, + ExecutionGraphInfoStore executionGraphInfoStore, + JobManagerRunnerFactory jobManagerRunnerFactory, + CleanupRunnerFactory cleanupRunnerFactory, + DispatcherBootstrapFactory dispatcherBootstrapFactory, + DispatcherOperationCaches dispatcherOperationCaches, + JobManagerRunnerRegistry jobManagerRunnerRegistry, + ResourceCleanerFactory resourceCleanerFactory) + throws Exception { + super( + rpcService, + fencingToken, + recoveredJobs, + recoveredDirtyJobs, + configuration, + highAvailabilityServices, + resourceManagerGatewayRetriever, + heartbeatServices, + blobServer, + fatalErrorHandler, + jobGraphWriter, + jobResultStore, + jobManagerMetricGroup, + metricServiceQueryAddress, + ioExecutor, + historyServerArchivist, + executionGraphInfoStore, + jobManagerRunnerFactory, + cleanupRunnerFactory, + dispatcherBootstrapFactory, + dispatcherOperationCaches, + jobManagerRunnerRegistry, + resourceCleanerFactory); + + this.startFuture = new CompletableFuture<>(); + } + @Override public void onStart() throws Exception { try { @@ -65,7 +147,14 @@ public void onStart() throws Exception { } void completeJobExecution(ExecutionGraphInfo executionGraphInfo) { - runAsync(() -> jobReachedTerminalState(executionGraphInfo)); + runAsync( + () -> { + try { + jobReachedTerminalState(executionGraphInfo); + } catch (Exception e) { + throw new CompletionException(e); + } + }); } CompletableFuture getJobTerminationFuture(@Nonnull JobID jobId, @Nonnull Time timeout) { @@ -80,4 +169,218 @@ CompletableFuture getNumberJobs(Time timeout) { void waitUntilStarted() { startFuture.join(); } + + public static TestingDispatcher.Builder builder() { + return new Builder(); + } + + public static class Builder { + private RpcService rpcService = new TestingRpcService(); + private DispatcherId fencingToken = DispatcherId.generate(); + private Collection recoveredJobs = Collections.emptyList(); + private Collection recoveredDirtyJobs = Collections.emptyList(); + private HighAvailabilityServices highAvailabilityServices = + new TestingHighAvailabilityServices(); + + private TestingResourceManagerGateway resourceManagerGateway = + new TestingResourceManagerGateway(); + private GatewayRetriever resourceManagerGatewayRetriever = + () -> CompletableFuture.completedFuture(resourceManagerGateway); + private HeartbeatServices heartbeatServices = new HeartbeatServices(1000L, 1000L); + + private JobGraphWriter jobGraphWriter = NoOpJobGraphWriter.INSTANCE; + private JobResultStore jobResultStore = new EmbeddedJobResultStore(); + + private Configuration configuration = new Configuration(); + + // even-though it's labeled as @Nullable, it's a mandatory field that needs to be set before + // building the Dispatcher instance + @Nullable private BlobServer blobServer = null; + private FatalErrorHandler fatalErrorHandler = new TestingFatalErrorHandler(); + private JobManagerMetricGroup jobManagerMetricGroup = + UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(); + @Nullable private String metricServiceQueryAddress = null; + private Executor ioExecutor = ForkJoinPool.commonPool(); + private HistoryServerArchivist historyServerArchivist = VoidHistoryServerArchivist.INSTANCE; + private ExecutionGraphInfoStore executionGraphInfoStore = + new MemoryExecutionGraphInfoStore(); + private JobManagerRunnerFactory jobManagerRunnerFactory = + new TestingJobMasterServiceLeadershipRunnerFactory(); + private CleanupRunnerFactory cleanupRunnerFactory = new TestingCleanupRunnerFactory(); + private DispatcherBootstrapFactory dispatcherBootstrapFactory = + (dispatcher, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(); + private DispatcherOperationCaches dispatcherOperationCaches = + new DispatcherOperationCaches(); + private JobManagerRunnerRegistry jobManagerRunnerRegistry = new JobManagerRunnerRegistry(1); + @Nullable private ResourceCleanerFactory resourceCleanerFactory; + + public Builder withRpcService(RpcService rpcService) { + this.rpcService = rpcService; + return this; + } + + public Builder withFencingToken(DispatcherId fencingToken) { + this.fencingToken = fencingToken; + return this; + } + + public Builder withRecoveredJobs(Collection recoveredJobs) { + this.recoveredJobs = recoveredJobs; + return this; + } + + public Builder withRecoveredDirtyJobs(Collection recoveredDirtyJobs) { + this.recoveredDirtyJobs = recoveredDirtyJobs; + return this; + } + + public Builder withHighAvailabilityServices( + HighAvailabilityServices highAvailabilityServices) { + this.highAvailabilityServices = highAvailabilityServices; + return this; + } + + public Builder withResourceManagerGateway( + TestingResourceManagerGateway resourceManagerGateway) { + this.resourceManagerGateway = resourceManagerGateway; + return this; + } + + public Builder withResourceManagerGatewayRetriever( + GatewayRetriever resourceManagerGatewayRetriever) { + this.resourceManagerGatewayRetriever = resourceManagerGatewayRetriever; + return this; + } + + public Builder withHeartbeatServices(HeartbeatServices heartbeatServices) { + this.heartbeatServices = heartbeatServices; + return this; + } + + public Builder withJobGraphWriter(JobGraphWriter jobGraphWriter) { + this.jobGraphWriter = jobGraphWriter; + return this; + } + + public Builder withJobResultStore(JobResultStore jobResultStore) { + this.jobResultStore = jobResultStore; + return this; + } + + public Builder withConfiguration(Configuration configuration) { + this.configuration = configuration; + return this; + } + + public Builder withBlobServer(BlobServer blobServer) { + this.blobServer = blobServer; + return this; + } + + public Builder withFatalErrorHandler(FatalErrorHandler fatalErrorHandler) { + this.fatalErrorHandler = fatalErrorHandler; + return this; + } + + public Builder withJobManagerMetricGroup(JobManagerMetricGroup jobManagerMetricGroup) { + this.jobManagerMetricGroup = jobManagerMetricGroup; + return this; + } + + public Builder withMetricServiceQueryAddress(@Nullable String metricServiceQueryAddress) { + this.metricServiceQueryAddress = metricServiceQueryAddress; + return this; + } + + public Builder withIoExecutor(Executor ioExecutor) { + this.ioExecutor = ioExecutor; + return this; + } + + public Builder withHistoryServerArchivist(HistoryServerArchivist historyServerArchivist) { + this.historyServerArchivist = historyServerArchivist; + return this; + } + + public Builder withExecutionGraphInfoStore( + ExecutionGraphInfoStore executionGraphInfoStore) { + this.executionGraphInfoStore = executionGraphInfoStore; + return this; + } + + public Builder withJobManagerRunnerFactory( + JobManagerRunnerFactory jobManagerRunnerFactory) { + this.jobManagerRunnerFactory = jobManagerRunnerFactory; + return this; + } + + public Builder withCleanupRunnerFactory(CleanupRunnerFactory cleanupRunnerFactory) { + this.cleanupRunnerFactory = cleanupRunnerFactory; + return this; + } + + public Builder withDispatcherBootstrapFactory( + DispatcherBootstrapFactory dispatcherBootstrapFactory) { + this.dispatcherBootstrapFactory = dispatcherBootstrapFactory; + return this; + } + + public Builder withDispatcherOperationCaches( + DispatcherOperationCaches dispatcherOperationCaches) { + this.dispatcherOperationCaches = dispatcherOperationCaches; + return this; + } + + public Builder withJobManagerRunnerRegistry( + JobManagerRunnerRegistry jobManagerRunnerRegistry) { + this.jobManagerRunnerRegistry = jobManagerRunnerRegistry; + return this; + } + + public Builder withResourceCleanerFactory(ResourceCleanerFactory resourceCleanerFactory) { + this.resourceCleanerFactory = resourceCleanerFactory; + return this; + } + + private ResourceCleanerFactory createDefaultResourceCleanerFactory() { + return new DispatcherResourceCleanerFactory( + ioExecutor, + jobManagerRunnerRegistry, + jobGraphWriter, + blobServer, + highAvailabilityServices, + jobManagerMetricGroup); + } + + public TestingDispatcher build() throws Exception { + return new TestingDispatcher( + rpcService, + fencingToken, + recoveredJobs, + recoveredDirtyJobs, + configuration, + highAvailabilityServices, + resourceManagerGatewayRetriever, + heartbeatServices, + Preconditions.checkNotNull( + blobServer, + "No BlobServer is specified for building the TestingDispatcher"), + fatalErrorHandler, + jobGraphWriter, + jobResultStore, + jobManagerMetricGroup, + metricServiceQueryAddress, + ioExecutor, + historyServerArchivist, + executionGraphInfoStore, + jobManagerRunnerFactory, + cleanupRunnerFactory, + dispatcherBootstrapFactory, + dispatcherOperationCaches, + jobManagerRunnerRegistry, + resourceCleanerFactory != null + ? resourceCleanerFactory + : createDefaultResourceCleanerFactory()); + } + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java index 384444d06193e..4a70baae513ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobManagerRunnerFactory.java @@ -18,19 +18,10 @@ package org.apache.flink.runtime.dispatcher; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.heartbeat.HeartbeatServices; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; +import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; -import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; -import org.apache.flink.runtime.rpc.FatalErrorHandler; -import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.util.Preconditions; -import javax.annotation.Nonnull; - import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.atomic.AtomicInteger; @@ -39,46 +30,30 @@ * Testing implementation of {@link JobManagerRunnerFactory} which returns a {@link * TestingJobManagerRunner}. */ -public class TestingJobManagerRunnerFactory implements JobManagerRunnerFactory { +public class TestingJobManagerRunnerFactory { private final BlockingQueue createdJobManagerRunner = new ArrayBlockingQueue<>(16); private final AtomicInteger numBlockingJobManagerRunners; - public TestingJobManagerRunnerFactory() { - this(0); - } - - public TestingJobManagerRunnerFactory(int numBlockingJobManagerRunners) { + protected TestingJobManagerRunnerFactory(int numBlockingJobManagerRunners) { this.numBlockingJobManagerRunners = new AtomicInteger(numBlockingJobManagerRunners); } - @Override - public TestingJobManagerRunner createJobManagerRunner( - JobGraph jobGraph, - Configuration configuration, - RpcService rpcService, - HighAvailabilityServices highAvailabilityServices, - HeartbeatServices heartbeatServices, - JobManagerSharedServices jobManagerServices, - JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, - FatalErrorHandler fatalErrorHandler, - long initializationTimestamp) - throws Exception { + protected TestingJobManagerRunner offerTestingJobManagerRunner(JobID jobId) { final TestingJobManagerRunner testingJobManagerRunner = - createTestingJobManagerRunner(jobGraph); + createTestingJobManagerRunner(jobId); Preconditions.checkState( createdJobManagerRunner.offer(testingJobManagerRunner), "Unable to persist created the new runner."); return testingJobManagerRunner; } - @Nonnull - private TestingJobManagerRunner createTestingJobManagerRunner(JobGraph jobGraph) { + private TestingJobManagerRunner createTestingJobManagerRunner(JobID jobId) { final boolean blockingTermination = numBlockingJobManagerRunners.getAndDecrement() > 0; return TestingJobManagerRunner.newBuilder() - .setJobId(jobGraph.getJobID()) + .setJobId(jobId) .setBlockingTermination(blockingTermination) .build(); } @@ -86,4 +61,8 @@ private TestingJobManagerRunner createTestingJobManagerRunner(JobGraph jobGraph) public TestingJobManagerRunner takeCreatedJobManagerRunner() throws InterruptedException { return createdJobManagerRunner.take(); } + + public int getQueueSize() { + return createdJobManagerRunner.size(); + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobMasterServiceLeadershipRunnerFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobMasterServiceLeadershipRunnerFactory.java new file mode 100644 index 0000000000000..0eb6d4c427052 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingJobMasterServiceLeadershipRunnerFactory.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmaster.JobManagerSharedServices; +import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; +import org.apache.flink.runtime.jobmaster.factories.JobManagerJobMetricGroupFactory; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.rpc.RpcService; + +/** + * {@code TestingJobMasterServiceLeadershipRunnerFactory} implements {@code JobManagerRunnerFactory} + * providing a factory method usually used for {@link + * org.apache.flink.runtime.jobmaster.JobMasterServiceLeadershipRunner} creations. + */ +public class TestingJobMasterServiceLeadershipRunnerFactory extends TestingJobManagerRunnerFactory + implements JobManagerRunnerFactory { + + public TestingJobMasterServiceLeadershipRunnerFactory() { + this(0); + } + + public TestingJobMasterServiceLeadershipRunnerFactory(int numBlockingJobManagerRunners) { + super(numBlockingJobManagerRunners); + } + + @Override + public TestingJobManagerRunner createJobManagerRunner( + JobGraph jobGraph, + Configuration configuration, + RpcService rpcService, + HighAvailabilityServices highAvailabilityServices, + HeartbeatServices heartbeatServices, + JobManagerSharedServices jobManagerServices, + JobManagerJobMetricGroupFactory jobManagerJobMetricGroupFactory, + FatalErrorHandler fatalErrorHandler, + long initializationTimestamp) + throws Exception { + return offerTestingJobManagerRunner(jobGraph.getJobID()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingPartialDispatcherServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingPartialDispatcherServices.java new file mode 100644 index 0000000000000..ba986291aabec --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingPartialDispatcherServices.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.BlobStore; +import org.apache.flink.runtime.blob.TestingBlobStoreBuilder; +import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; +import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler; +import org.apache.flink.runtime.rpc.FatalErrorHandler; +import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; + +import javax.annotation.Nullable; + +import java.io.File; +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; + +/** + * {@code TestingPartialDispatcherServices} implements {@link PartialDispatcherServices} to be used + * in test contexts. + */ +public class TestingPartialDispatcherServices extends PartialDispatcherServices { + public TestingPartialDispatcherServices( + Configuration configuration, + HighAvailabilityServices highAvailabilityServices, + GatewayRetriever resourceManagerGatewayRetriever, + BlobServer blobServer, + HeartbeatServices heartbeatServices, + JobManagerMetricGroupFactory jobManagerMetricGroupFactory, + ExecutionGraphInfoStore executionGraphInfoStore, + FatalErrorHandler fatalErrorHandler, + HistoryServerArchivist historyServerArchivist, + @Nullable String metricQueryServiceAddress, + Executor ioExecutor, + DispatcherOperationCaches operationCaches) { + super( + configuration, + highAvailabilityServices, + resourceManagerGatewayRetriever, + blobServer, + heartbeatServices, + jobManagerMetricGroupFactory, + executionGraphInfoStore, + fatalErrorHandler, + historyServerArchivist, + metricQueryServiceAddress, + ioExecutor, + operationCaches); + } + + public static Builder builder() { + return new Builder(); + } + + /** {@code Builder} for creating {@code TestingPartialDispatcherServices} instances. */ + public static class Builder { + + private HighAvailabilityServices highAvailabilityServices = + new TestingHighAvailabilityServicesBuilder().build(); + private GatewayRetriever resourceManagerGatewayRetriever = + CompletableFuture::new; + private BlobStore blobStore = new TestingBlobStoreBuilder().createTestingBlobStore(); + private HeartbeatServices heartbeatServices = new TestingHeartbeatServices(); + private JobManagerMetricGroupFactory jobManagerMetricGroupFactory = + UnregisteredMetricGroups::createUnregisteredJobManagerMetricGroup; + private ExecutionGraphInfoStore executionGraphInfoStore = + new MemoryExecutionGraphInfoStore(); + private FatalErrorHandler fatalErrorHandler = NoOpFatalErrorHandler.INSTANCE; + private HistoryServerArchivist historyServerArchivist = VoidHistoryServerArchivist.INSTANCE; + @Nullable private String metricQueryServiceAddress = null; + private DispatcherOperationCaches operationCaches = new DispatcherOperationCaches(); + private Executor ioExecutor = ForkJoinPool.commonPool(); + + private Builder() {} + + public Builder withHighAvailabilityServices( + HighAvailabilityServices highAvailabilityServices) { + this.highAvailabilityServices = highAvailabilityServices; + return this; + } + + public Builder withResourceManagerGatewayRetriever( + GatewayRetriever resourceManagerGatewayRetriever) { + this.resourceManagerGatewayRetriever = resourceManagerGatewayRetriever; + return this; + } + + public Builder withBlobStore(BlobStore blobStore) { + this.blobStore = blobStore; + return this; + } + + public Builder withHeartbeatServices(HeartbeatServices heartbeatServices) { + this.heartbeatServices = heartbeatServices; + return this; + } + + public Builder withJobManagerMetricGroupFactory( + JobManagerMetricGroupFactory jobManagerMetricGroupFactory) { + this.jobManagerMetricGroupFactory = jobManagerMetricGroupFactory; + return this; + } + + public Builder withExecutionGraphInfoStore( + ExecutionGraphInfoStore executionGraphInfoStore) { + this.executionGraphInfoStore = executionGraphInfoStore; + return this; + } + + public Builder withFatalErrorHandler(FatalErrorHandler fatalErrorHandler) { + this.fatalErrorHandler = fatalErrorHandler; + return this; + } + + public Builder withHistoryServerArchivist(HistoryServerArchivist historyServerArchivist) { + this.historyServerArchivist = historyServerArchivist; + return this; + } + + public Builder withMetricQueryServiceAddress(@Nullable String metricQueryServiceAddress) { + this.metricQueryServiceAddress = metricQueryServiceAddress; + return this; + } + + public Builder withOperationCaches(DispatcherOperationCaches operationCaches) { + this.operationCaches = operationCaches; + return this; + } + + public Builder withIoExecutor(Executor ioExecutor) { + this.ioExecutor = ioExecutor; + return this; + } + + public TestingPartialDispatcherServices build(File storageDir, Configuration configuration) + throws IOException { + try (BlobServer blobServer = new BlobServer(configuration, storageDir, blobStore)) { + return build(blobServer, configuration); + } + } + + public TestingPartialDispatcherServices build( + BlobServer blobServer, Configuration configuration) { + return new TestingPartialDispatcherServices( + configuration, + highAvailabilityServices, + resourceManagerGatewayRetriever, + blobServer, + heartbeatServices, + jobManagerMetricGroupFactory, + executionGraphInfoStore, + fatalErrorHandler, + historyServerArchivist, + metricQueryServiceAddress, + ioExecutor, + operationCaches); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerTest.java new file mode 100644 index 0000000000000..0e16bd43b98c2 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerTest.java @@ -0,0 +1,589 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.CheckpointsCleaner; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; +import org.apache.flink.runtime.checkpoint.TestingCheckpointIDCounter; +import org.apache.flink.runtime.checkpoint.TestingCheckpointRecoveryFactory; +import org.apache.flink.runtime.checkpoint.TestingCompletedCheckpointStore; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.dispatcher.UnavailableDispatcherOperationException; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.ErrorInfo; +import org.apache.flink.runtime.jobmaster.JobManagerRunnerResult; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.runtime.state.SharedStateRegistry; +import org.apache.flink.runtime.state.SharedStateRegistryFactory; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.SerializedThrowable; +import org.apache.flink.util.concurrent.Executors; +import org.apache.flink.util.function.ThrowingConsumer; + +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * {@code CheckpointResourcesCleanupRunnerTest} tests the {@link CheckpointResourcesCleanupRunner} + * implementation. + */ +public class CheckpointResourcesCleanupRunnerTest { + + private static final ThrowingConsumer + BEFORE_START = ignored -> {}; + private static final ThrowingConsumer + AFTER_START = CheckpointResourcesCleanupRunner::start; + private static final ThrowingConsumer + AFTER_CLOSE = + runner -> { + runner.start(); + runner.close(); + }; + + @Test + public void testIsInitializedBeforeStart() throws Exception { + testIsInitialized(BEFORE_START); + } + + @Test + public void testIsInitializedAfterStart() throws Exception { + testIsInitialized(AFTER_START); + } + + @Test + public void testIsInitializedAfterClose() throws Exception { + testIsInitialized(AFTER_CLOSE); + } + + private static void testIsInitialized( + ThrowingConsumer + preCheckLifecycleHandling) + throws Exception { + final CheckpointResourcesCleanupRunner testInstance = new TestInstanceBuilder().build(); + preCheckLifecycleHandling.accept(testInstance); + + assertThat(testInstance.isInitialized()).isTrue(); + } + + @Test + public void testCloseAsyncBeforeStart() { + final CheckpointResourcesCleanupRunner testInstance = new TestInstanceBuilder().build(); + assertThat(testInstance.closeAsync()).isNotCompleted(); + } + + @Test + public void testSuccessfulCloseAsyncAfterStart() throws Exception { + final CompletableFuture completedCheckpointStoreShutdownFuture = + new CompletableFuture<>(); + final CompletableFuture checkpointIdCounterShutdownFuture = + new CompletableFuture<>(); + + final HaltingCheckpointRecoveryFactory checkpointRecoveryFactory = + new HaltingCheckpointRecoveryFactory( + completedCheckpointStoreShutdownFuture, checkpointIdCounterShutdownFuture); + final CheckpointResourcesCleanupRunner testInstance = + new TestInstanceBuilder() + .withCheckpointRecoveryFactory(checkpointRecoveryFactory) + .withExecutor(ForkJoinPool.commonPool()) + .build(); + testInstance.start(); + + assertThat(completedCheckpointStoreShutdownFuture) + .as("The CompletedCheckpointStore shouldn't have been shut down, yet.") + .isNotCompleted(); + assertThat(checkpointIdCounterShutdownFuture) + .as("The CheckpointIDCounter shouldn't have been shut down, yet.") + .isNotCompleted(); + + assertThat(testInstance.closeAsync()) + .as( + "closeAsync shouldn't have been completed, yet, since the shutdown of the components is not completed.") + .isNotCompleted(); + + checkpointRecoveryFactory.triggerCreation(); + + assertThat(completedCheckpointStoreShutdownFuture) + .as("The CompletedCheckpointStore should have been shut down properly.") + .succeedsWithin(Duration.ofMillis(100)) + .isEqualTo(JobStatus.FINISHED); + assertThat(checkpointIdCounterShutdownFuture) + .as("The CheckpointIDCounter should have been shut down properly.") + .succeedsWithin(Duration.ofMillis(100)) + .isEqualTo(JobStatus.FINISHED); + + assertThat(testInstance.closeAsync()).succeedsWithin(Duration.ofMillis(100)); + } + + @Test + public void testCloseAsyncAfterStartAndErrorInCompletedCheckpointStoreShutdown() + throws Exception { + final CompletableFuture checkpointIdCounterShutdownFuture = + new CompletableFuture<>(); + + final HaltingCheckpointRecoveryFactory checkpointRecoveryFactory = + new HaltingCheckpointRecoveryFactory( + TestingCompletedCheckpointStore.builder() + .withShutdownConsumer( + (ignoredJobStatus, ignoredCheckpointsCleaner) -> { + throw new RuntimeException( + "Expected RuntimeException simulating an error during shutdown."); + }) + .build(), + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + checkpointIdCounterShutdownFuture)); + final CheckpointResourcesCleanupRunner testInstance = + new TestInstanceBuilder() + .withCheckpointRecoveryFactory(checkpointRecoveryFactory) + .withExecutor(ForkJoinPool.commonPool()) + .build(); + testInstance.start(); + + assertThat(checkpointIdCounterShutdownFuture) + .as("The CheckpointIDCounter shouldn't have been shut down, yet.") + .isNotCompleted(); + + assertThat(testInstance.closeAsync()) + .as( + "closeAsync shouldn't have been completed, yet, since the shutdown of the components is not completed.") + .isNotCompleted(); + + checkpointRecoveryFactory.triggerCreation(); + + assertThat(checkpointIdCounterShutdownFuture) + .as("The CheckpointIDCounter should have been shut down properly.") + .succeedsWithin(Duration.ofMillis(100)) + .isEqualTo(JobStatus.FINISHED); + + assertThat(testInstance.closeAsync()) + .failsWithin(Duration.ofMillis(100)) + .withThrowableOfType(ExecutionException.class) + .withCauseInstanceOf(RuntimeException.class); + } + + @Test + public void testCloseAsyncAfterStartAndErrorInCheckpointIDCounterShutdown() throws Exception { + final CompletableFuture completedCheckpointStoreShutdownFuture = + new CompletableFuture<>(); + + final HaltingCheckpointRecoveryFactory checkpointRecoveryFactory = + new HaltingCheckpointRecoveryFactory( + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints( + completedCheckpointStoreShutdownFuture), + TestingCheckpointIDCounter.builder() + .withShutdownConsumer( + ignoredJobStatus -> { + throw new RuntimeException( + "Expected RuntimeException simulating an error during shutdown."); + }) + .build()); + final CheckpointResourcesCleanupRunner testInstance = + new TestInstanceBuilder() + .withCheckpointRecoveryFactory(checkpointRecoveryFactory) + .withExecutor(ForkJoinPool.commonPool()) + .build(); + testInstance.start(); + + assertThat(completedCheckpointStoreShutdownFuture) + .as("The CompletedCheckpointStore shouldn't have been shut down, yet.") + .isNotCompleted(); + + assertThat(testInstance.closeAsync()) + .as( + "closeAsync shouldn't have been completed, yet, since the shutdown of the components is not completed.") + .isNotCompleted(); + + checkpointRecoveryFactory.triggerCreation(); + + assertThat(completedCheckpointStoreShutdownFuture) + .as("The CompletedCheckpointStore should have been shut down properly.") + .succeedsWithin(Duration.ofMillis(100)) + .isEqualTo(JobStatus.FINISHED); + + assertThat(testInstance.closeAsync()) + .failsWithin(Duration.ofMillis(100)) + .withThrowableOfType(ExecutionException.class) + .withCauseInstanceOf(RuntimeException.class); + } + + @Test + public void testResultFutureWithSuccessBeforeStart() throws Exception { + testResultFutureWithSuccess(BEFORE_START); + } + + @Test + public void testResultFutureWithSuccessAfterStart() throws Exception { + testResultFutureWithSuccess(AFTER_START); + } + + @Test + public void testResultFutureWithSuccessAfterClose() throws Exception { + testResultFutureWithSuccess(AFTER_CLOSE); + } + + private static void testResultFutureWithSuccess( + ThrowingConsumer + preCheckLifecycleHandling) + throws Exception { + testResultFuture(createDummySuccessJobResult(), preCheckLifecycleHandling); + } + + @Test + public void testResultFutureWithErrorBeforeStart() throws Exception { + testResultFutureWithError(BEFORE_START); + } + + @Test + public void testResultFutureWithErrorAfterStart() throws Exception { + testResultFutureWithError(AFTER_START); + } + + @Test + public void testResultFutureWithErrorAfterClose() throws Exception { + testResultFutureWithError(AFTER_CLOSE); + } + + private static void testResultFutureWithError( + ThrowingConsumer + preCheckLifecycleHandling) + throws Exception { + final SerializedThrowable expectedError = + new SerializedThrowable(new Exception("Expected exception")); + final CompletableFuture actualResult = + testResultFuture( + createJobResultWithFailure(expectedError), preCheckLifecycleHandling); + + assertThat(actualResult) + .succeedsWithin(Duration.ZERO) + .extracting(JobManagerRunnerResult::getExecutionGraphInfo) + .extracting(ExecutionGraphInfo::getArchivedExecutionGraph) + .extracting(AccessExecutionGraph::getFailureInfo) + .extracting(ErrorInfo::getException) + .isEqualTo(expectedError); + } + + private static CompletableFuture testResultFuture( + JobResult jobResult, + ThrowingConsumer + preCheckLifecycleHandling) + throws Exception { + final CheckpointResourcesCleanupRunner testInstance = + new TestInstanceBuilder().withJobResult(jobResult).build(); + preCheckLifecycleHandling.accept(testInstance); + + assertThat(testInstance.getResultFuture()) + .succeedsWithin(Duration.ZERO) + .extracting(JobManagerRunnerResult::isSuccess) + .isEqualTo(true); + + return testInstance.getResultFuture(); + } + + @Test + public void testGetJobID() { + final JobID jobId = new JobID(); + final CheckpointResourcesCleanupRunner testInstance = + new TestInstanceBuilder() + .withJobResult(createJobResult(jobId, ApplicationStatus.CANCELED)) + .build(); + assertThat(testInstance.getJobID()).isEqualTo(jobId); + } + + @Test + public void testGetJobMasterGatewayBeforeStart() throws Exception { + testGetJobMasterGateway(BEFORE_START); + } + + @Test + public void testGetJobMasterGatewayAfterStart() throws Exception { + testGetJobMasterGateway(AFTER_START); + } + + @Test + public void testGetJobMasterGatewayAfterClose() throws Exception { + testGetJobMasterGateway(AFTER_CLOSE); + } + + private static void testGetJobMasterGateway( + ThrowingConsumer + preCheckLifecycleHandling) + throws Exception { + final CheckpointResourcesCleanupRunner testInstance = new TestInstanceBuilder().build(); + preCheckLifecycleHandling.accept(testInstance); + + assertThat(testInstance.getJobMasterGateway()) + .failsWithin(Duration.ZERO) + .withThrowableOfType(ExecutionException.class) + .withCauseInstanceOf(UnavailableDispatcherOperationException.class); + } + + @Test + public void testRequestJob_ExceptionHistory() { + testRequestJob( + createDummySuccessJobResult(), + System.currentTimeMillis(), + actualExecutionGraphInfo -> + assertThat(actualExecutionGraphInfo) + .extracting(ExecutionGraphInfo::getExceptionHistory) + .asList() + .isEmpty()); + } + + @Test + public void testRequestJob_JobName() { + testRequestJobExecutionGraph( + createDummySuccessJobResult(), + System.currentTimeMillis(), + actualExecutionGraph -> + assertThat(actualExecutionGraph) + .extracting(AccessExecutionGraph::getJobName) + .isEqualTo("unknown")); + } + + @Test + public void testRequestJob_JobId() { + final JobResult jobResult = createDummySuccessJobResult(); + testRequestJobExecutionGraph( + jobResult, + System.currentTimeMillis(), + actualExecutionGraph -> + assertThat(actualExecutionGraph) + .extracting(AccessExecutionGraph::getJobID) + .isEqualTo(jobResult.getJobId())); + } + + @Test + public void testRequestJob_JobState() { + final JobResult jobResult = createDummySuccessJobResult(); + testRequestJobExecutionGraph( + jobResult, + System.currentTimeMillis(), + actualExecutionGraph -> + assertThat(actualExecutionGraph) + .extracting(AccessExecutionGraph::getState) + .isEqualTo(jobResult.getApplicationStatus().deriveJobStatus())); + } + + @Test + public void testRequestJob_InitiatizationTimestamp() { + final long initializationTimestamp = System.currentTimeMillis(); + testRequestJobExecutionGraph( + createDummySuccessJobResult(), + initializationTimestamp, + actualExecutionGraph -> + assertThat(actualExecutionGraph.getStatusTimestamp(JobStatus.INITIALIZING)) + .isEqualTo(initializationTimestamp)); + } + + @Test + public void testRequestJobWithFailure() { + final SerializedThrowable expectedError = + new SerializedThrowable(new Exception("Expected exception")); + final JobResult jobResult = createJobResultWithFailure(expectedError); + testRequestJobExecutionGraph( + jobResult, + System.currentTimeMillis(), + actualExecutionGraph -> + assertThat(actualExecutionGraph) + .extracting(AccessExecutionGraph::getFailureInfo) + .extracting(ErrorInfo::getException) + .isEqualTo(expectedError)); + } + + private static void testRequestJobExecutionGraph( + JobResult jobResult, + long initializationTimestamp, + ThrowingConsumer assertion) { + testRequestJob( + jobResult, + initializationTimestamp, + actualExecutionGraphInfo -> + assertThat(actualExecutionGraphInfo) + .extracting(ExecutionGraphInfo::getArchivedExecutionGraph) + .satisfies(assertion::accept)); + } + + private static void testRequestJob( + JobResult jobResult, + long initializationTimestamp, + ThrowingConsumer assertion) { + final CheckpointResourcesCleanupRunner testInstance = + new TestInstanceBuilder() + .withJobResult(jobResult) + .withInitializationTimestamp(initializationTimestamp) + .build(); + + final CompletableFuture response = + testInstance.requestJob(Time.milliseconds(0)); + assertThat(response).succeedsWithin(Duration.ZERO).satisfies(assertion::accept); + } + + private static JobResult createDummySuccessJobResult() { + return createJobResult(new JobID(), ApplicationStatus.SUCCEEDED); + } + + private static JobResult createJobResultWithFailure(SerializedThrowable throwable) { + return new JobResult.Builder() + .jobId(new JobID()) + .applicationStatus(ApplicationStatus.FAILED) + .serializedThrowable(throwable) + .netRuntime(1) + .build(); + } + + private static JobResult createJobResult(JobID jobId, ApplicationStatus applicationStatus) { + return new JobResult.Builder() + .jobId(jobId) + .applicationStatus(applicationStatus) + .netRuntime(1) + .build(); + } + + private static CheckpointRecoveryFactory createCheckpointRecoveryFactory() { + return new TestingCheckpointRecoveryFactory( + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints( + new CompletableFuture<>()), + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + new CompletableFuture<>())); + } + + private static class TestInstanceBuilder { + + private JobResult jobResult = createDummySuccessJobResult(); + private CheckpointRecoveryFactory checkpointRecoveryFactory = + createCheckpointRecoveryFactory(); + private CheckpointsCleaner checkpointsCleaner = new CheckpointsCleaner(); + private SharedStateRegistryFactory sharedStateRegistryFactory = + SharedStateRegistry.DEFAULT_FACTORY; + private Executor executor = Executors.directExecutor(); + private Configuration configuration = new Configuration(); + private long initializationTimestamp = System.currentTimeMillis(); + + public TestInstanceBuilder withJobResult(JobResult jobResult) { + this.jobResult = jobResult; + return this; + } + + public TestInstanceBuilder withCheckpointRecoveryFactory( + CheckpointRecoveryFactory checkpointRecoveryFactory) { + this.checkpointRecoveryFactory = checkpointRecoveryFactory; + return this; + } + + public TestInstanceBuilder withCheckpointsCleaner(CheckpointsCleaner checkpointsCleaner) { + this.checkpointsCleaner = checkpointsCleaner; + return this; + } + + public TestInstanceBuilder withSharedStateRegistryFactory( + SharedStateRegistryFactory sharedStateRegistryFactory) { + this.sharedStateRegistryFactory = sharedStateRegistryFactory; + return this; + } + + public TestInstanceBuilder withExecutor(Executor executor) { + this.executor = executor; + return this; + } + + public TestInstanceBuilder withConfiguration(Configuration configuration) { + this.configuration = configuration; + return this; + } + + public TestInstanceBuilder withInitializationTimestamp(long initializationTimestamp) { + this.initializationTimestamp = initializationTimestamp; + return this; + } + + public CheckpointResourcesCleanupRunner build() { + return new CheckpointResourcesCleanupRunner( + jobResult, + checkpointRecoveryFactory, + checkpointsCleaner, + sharedStateRegistryFactory, + configuration, + executor, + initializationTimestamp); + } + } + + private static class HaltingCheckpointRecoveryFactory implements CheckpointRecoveryFactory { + + private final CompletedCheckpointStore completedCheckpointStore; + private final CheckpointIDCounter checkpointIDCounter; + + private final OneShotLatch creationLatch = new OneShotLatch(); + + public HaltingCheckpointRecoveryFactory( + CompletableFuture completableCheckpointStoreShutDownFuture, + CompletableFuture checkpointIDCounterShutDownFuture) { + this( + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints( + completableCheckpointStoreShutDownFuture), + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + checkpointIDCounterShutDownFuture)); + } + + public HaltingCheckpointRecoveryFactory( + CompletedCheckpointStore completedCheckpointStore, + CheckpointIDCounter checkpointIDCounter) { + this.completedCheckpointStore = Preconditions.checkNotNull(completedCheckpointStore); + this.checkpointIDCounter = Preconditions.checkNotNull(checkpointIDCounter); + } + + @Override + public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( + JobID jobId, + int maxNumberOfCheckpointsToRetain, + SharedStateRegistryFactory sharedStateRegistryFactory, + Executor ioExecutor) + throws Exception { + creationLatch.await(); + return completedCheckpointStore; + } + + @Override + public CheckpointIDCounter createCheckpointIDCounter(JobID jobId) throws Exception { + creationLatch.await(); + return checkpointIDCounter; + } + + public void triggerCreation() { + creationLatch.trigger(); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultResourceCleanerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultResourceCleanerTest.java new file mode 100644 index 0000000000000..47a29cbd74f6f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultResourceCleanerTest.java @@ -0,0 +1,227 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.concurrent.Executors; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.function.BiFunction; +import java.util.function.Consumer; + +import static org.assertj.core.api.Assertions.assertThat; + +/** {@code DefaultResourceCleanerTest} tests {@link DefaultResourceCleaner}. */ +public class DefaultResourceCleanerTest { + + private static final Executor EXECUTOR = Executors.directExecutor(); + private static final JobID JOB_ID = new JobID(); + + private DefaultResourceCleaner testInstance; + private CleanupCallback cleanup0; + private CleanupCallback cleanup1; + + @BeforeEach + public void setup() { + cleanup0 = CleanupCallback.withoutCompletionOnCleanup(); + cleanup1 = CleanupCallback.withoutCompletionOnCleanup(); + + testInstance = new DefaultResourceCleaner(EXECUTOR); + testInstance.withCleanupOf(cleanup0).withCleanupOf(cleanup1); + } + + @Test + public void testSuccessfulConcurrentCleanup() { + CompletableFuture cleanupResult = testInstance.cleanupAsync(JOB_ID); + + assertThat(cleanupResult).isNotCompleted(); + assertThat(cleanup0).extracting(CleanupCallback::getProcessedJobId).isEqualTo(JOB_ID); + assertThat(cleanup1).extracting(CleanupCallback::getProcessedJobId).isEqualTo(JOB_ID); + + cleanup0.completeCleanup(); + assertThat(cleanupResult).isNotCompleted(); + + cleanup1.completeCleanup(); + assertThat(cleanupResult).isCompleted(); + } + + @Test + public void testConcurrentCleanupWithExceptionFirst() { + CompletableFuture cleanupResult = testInstance.cleanupAsync(JOB_ID); + + assertThat(cleanupResult).isNotCompleted(); + assertThat(cleanup0).extracting(CleanupCallback::getProcessedJobId).isEqualTo(JOB_ID); + assertThat(cleanup1).extracting(CleanupCallback::getProcessedJobId).isEqualTo(JOB_ID); + + final RuntimeException expectedException = new RuntimeException("Expected exception"); + cleanup0.completeCleanupExceptionally(expectedException); + assertThat(cleanupResult).isNotCompleted(); + + cleanup1.completeCleanup(); + assertThat(cleanupResult) + .failsWithin(Duration.ZERO) + .withThrowableOfType(ExecutionException.class) + .withCause(expectedException); + } + + @Test + public void testConcurrentCleanupWithExceptionSecond() { + CompletableFuture cleanupResult = testInstance.cleanupAsync(JOB_ID); + + assertThat(cleanupResult).isNotCompleted(); + assertThat(cleanup0).extracting(CleanupCallback::getProcessedJobId).isEqualTo(JOB_ID); + assertThat(cleanup1).extracting(CleanupCallback::getProcessedJobId).isEqualTo(JOB_ID); + + cleanup0.completeCleanup(); + assertThat(cleanupResult).isNotCompleted(); + + final RuntimeException expectedException = new RuntimeException("Expected exception"); + cleanup1.completeCleanupExceptionally(expectedException); + assertThat(cleanupResult) + .failsWithin(Duration.ZERO) + .withThrowableOfType(ExecutionException.class) + .withCause(expectedException); + } + + @Test + public void testHighestPriorityCleanupBlocksAllOtherCleanups() { + final CleanupCallback highPriorityCleanup = CleanupCallback.withoutCompletionOnCleanup(); + final CleanupCallback lowerThanHighPriorityCleanup = + CleanupCallback.withCompletionOnCleanup(); + final CleanupCallback noPriorityCleanup0 = CleanupCallback.withCompletionOnCleanup(); + final CleanupCallback noPriorityCleanup1 = CleanupCallback.withCompletionOnCleanup(); + + final DefaultResourceCleaner testInstance = new DefaultResourceCleaner(EXECUTOR); + + testInstance + .withPriorityCleanupOf(highPriorityCleanup) + .withPriorityCleanupOf(lowerThanHighPriorityCleanup) + .withCleanupOf(noPriorityCleanup0) + .withPriorityCleanupOf(noPriorityCleanup1); + + final CompletableFuture overallCleanupResult = testInstance.cleanupAsync(JOB_ID); + + assertThat(highPriorityCleanup.isDone()).isFalse(); + assertThat(lowerThanHighPriorityCleanup.isDone()).isFalse(); + assertThat(noPriorityCleanup0.isDone()).isFalse(); + assertThat(noPriorityCleanup1.isDone()).isFalse(); + + assertThat(overallCleanupResult.isDone()).isFalse(); + + highPriorityCleanup.completeCleanup(); + + assertThat(overallCleanupResult).succeedsWithin(Duration.ofMillis(100)); + + assertThat(highPriorityCleanup.isDone()).isTrue(); + assertThat(lowerThanHighPriorityCleanup.isDone()).isTrue(); + assertThat(noPriorityCleanup0.isDone()).isTrue(); + assertThat(noPriorityCleanup1.isDone()).isTrue(); + } + + @Test + public void testMediumPriorityCleanupBlocksAllLowerPrioritizedCleanups() { + final CleanupCallback highPriorityCleanup = CleanupCallback.withCompletionOnCleanup(); + final CleanupCallback lowerThanHighPriorityCleanup = + CleanupCallback.withoutCompletionOnCleanup(); + final CleanupCallback noPriorityCleanup0 = CleanupCallback.withCompletionOnCleanup(); + final CleanupCallback noPriorityCleanup1 = CleanupCallback.withCompletionOnCleanup(); + + final DefaultResourceCleaner testInstance = new DefaultResourceCleaner(EXECUTOR); + + testInstance + .withPriorityCleanupOf(highPriorityCleanup) + .withPriorityCleanupOf(lowerThanHighPriorityCleanup) + .withCleanupOf(noPriorityCleanup0) + .withPriorityCleanupOf(noPriorityCleanup1); + + assertThat(highPriorityCleanup.isDone()).isFalse(); + + final CompletableFuture overallCleanupResult = testInstance.cleanupAsync(JOB_ID); + + assertThat(highPriorityCleanup.isDone()).isTrue(); + assertThat(lowerThanHighPriorityCleanup.isDone()).isFalse(); + assertThat(noPriorityCleanup0.isDone()).isFalse(); + assertThat(noPriorityCleanup1.isDone()).isFalse(); + + assertThat(overallCleanupResult.isDone()).isFalse(); + + lowerThanHighPriorityCleanup.completeCleanup(); + + assertThat(overallCleanupResult).succeedsWithin(Duration.ofMillis(100)); + + assertThat(highPriorityCleanup.isDone()).isTrue(); + assertThat(lowerThanHighPriorityCleanup.isDone()).isTrue(); + assertThat(noPriorityCleanup0.isDone()).isTrue(); + assertThat(noPriorityCleanup1.isDone()).isTrue(); + } + + private static class CleanupCallback + implements BiFunction> { + + private final CompletableFuture resultFuture = new CompletableFuture<>(); + private JobID jobId; + + private final Consumer> internalFunction; + + public static CleanupCallback withCompletionOnCleanup() { + return new CleanupCallback(resultFuture -> resultFuture.complete(null)); + } + + public static CleanupCallback withoutCompletionOnCleanup() { + return new CleanupCallback(ignoredResultFuture -> {}); + } + + private CleanupCallback(Consumer> internalFunction) { + this.internalFunction = internalFunction; + } + + @Override + public CompletableFuture apply(JobID jobId, Executor executor) { + Preconditions.checkState(this.jobId == null); + this.jobId = jobId; + + internalFunction.accept(resultFuture); + + return resultFuture; + } + + public boolean isDone() { + return resultFuture.isDone(); + } + + public JobID getProcessedJobId() { + return jobId; + } + + public void completeCleanup() { + this.resultFuture.complete(null); + } + + public void completeCleanupExceptionally(Throwable expectedException) { + this.resultFuture.completeExceptionally(expectedException); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactoryTest.java new file mode 100644 index 0000000000000..4b3ffbccc736d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherResourceCleanerFactoryTest.java @@ -0,0 +1,420 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.BlobStore; +import org.apache.flink.runtime.blob.TestingBlobStoreBuilder; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.dispatcher.JobManagerRunnerRegistry; +import org.apache.flink.runtime.highavailability.HighAvailabilityServices; +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmanager.JobGraphWriter; +import org.apache.flink.runtime.leaderelection.LeaderElectionService; +import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.metrics.MetricRegistry; +import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; +import org.apache.flink.runtime.metrics.util.TestingMetricRegistry; +import org.apache.flink.util.concurrent.Executors; +import org.apache.flink.util.concurrent.FutureUtils; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeoutException; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * {@code DispatcherResourceCleanerFactoryTest} verifies that the resources are properly cleaned up + * for both, the {@link GloballyCleanableResource} and {@link LocallyCleanableResource} of the + * {@link org.apache.flink.runtime.dispatcher.Dispatcher}. + */ +public class DispatcherResourceCleanerFactoryTest { + + private static final JobID JOB_ID = new JobID(); + + private CleanableJobManagerRegistry jobManagerRunnerRegistry; + private CleanableJobGraphWriter jobGraphWriter; + private CleanableBlobServer blobServer; + private CleanableHighAvailabilityServices highAvailabilityServices; + private JobManagerMetricGroup jobManagerMetricGroup; + + private DispatcherResourceCleanerFactory testInstance; + + @BeforeEach + public void setup() throws IOException { + jobManagerRunnerRegistry = new CleanableJobManagerRegistry(); + jobGraphWriter = new CleanableJobGraphWriter(); + blobServer = new CleanableBlobServer(); + highAvailabilityServices = new CleanableHighAvailabilityServices(); + + MetricRegistry metricRegistry = TestingMetricRegistry.builder().build(); + jobManagerMetricGroup = + JobManagerMetricGroup.createJobManagerMetricGroup( + metricRegistry, "ignored hostname"); + jobManagerMetricGroup.addJob(JOB_ID, "ignored job name"); + + testInstance = + new DispatcherResourceCleanerFactory( + Executors.directExecutor(), + jobManagerRunnerRegistry, + jobGraphWriter, + blobServer, + highAvailabilityServices, + jobManagerMetricGroup); + } + + @Test + public void testLocalResourceCleaning() { + assertGlobalCleanupNotTriggered(); + assertLocalCleanupNotTriggered(); + assertJobManagerMetricGroupNotCleaned(); + + final CompletableFuture cleanupResultFuture = + testInstance.createLocalResourceCleaner().cleanupAsync(JOB_ID); + + assertGlobalCleanupNotTriggered(); + assertLocalCleanupTriggeredWaitingForJobManagerRunnerRegistry(); + assertJobManagerMetricGroupNotCleaned(); + + assertThat(cleanupResultFuture).isNotCompleted(); + + jobManagerRunnerRegistry.completeLocalCleanup(); + + assertGlobalCleanupNotTriggered(); + assertLocalCleanupTriggered(); + assertJobManagerMetricGroupCleaned(); + + assertThat(cleanupResultFuture).isCompleted(); + } + + @Test + public void testGlobalResourceCleaning() + throws ExecutionException, InterruptedException, TimeoutException { + assertGlobalCleanupNotTriggered(); + assertLocalCleanupNotTriggered(); + assertJobManagerMetricGroupNotCleaned(); + + final CompletableFuture cleanupResultFuture = + testInstance.createGlobalResourceCleaner().cleanupAsync(JOB_ID); + + assertGlobalCleanupTriggeredWaitingForJobManagerRunnerRegistry(); + assertLocalCleanupNotTriggered(); + assertJobManagerMetricGroupNotCleaned(); + + assertThat(cleanupResultFuture).isNotCompleted(); + + jobManagerRunnerRegistry.completeGlobalCleanup(); + + assertGlobalCleanupTriggered(); + assertLocalCleanupNotTriggered(); + assertJobManagerMetricGroupCleaned(); + + assertThat(cleanupResultFuture).isCompleted(); + } + + private void assertLocalCleanupNotTriggered() { + assertThat(jobManagerRunnerRegistry.getLocalCleanupFuture()).isNotDone(); + assertThat(jobGraphWriter.getLocalCleanupFuture()).isNotDone(); + assertThat(blobServer.getLocalCleanupFuture()).isNotDone(); + assertThat(highAvailabilityServices.getLocalCleanupFuture()).isNotDone(); + } + + private void assertLocalCleanupTriggeredWaitingForJobManagerRunnerRegistry() { + assertThat(jobManagerRunnerRegistry.getLocalCleanupFuture()).isDone(); + + // the JobManagerRunnerRegistry needs to be cleaned up first + assertThat(jobGraphWriter.getLocalCleanupFuture()).isNotDone(); + assertThat(blobServer.getLocalCleanupFuture()).isNotDone(); + assertThat(highAvailabilityServices.getLocalCleanupFuture()).isNotDone(); + } + + private void assertGlobalCleanupNotTriggered() { + assertThat(jobManagerRunnerRegistry.getGlobalCleanupFuture()).isNotDone(); + assertThat(jobGraphWriter.getGlobalCleanupFuture()).isNotDone(); + assertThat(blobServer.getGlobalCleanupFuture()).isNotDone(); + assertThat(highAvailabilityServices.getGlobalCleanupFuture()).isNotDone(); + } + + private void assertGlobalCleanupTriggeredWaitingForJobManagerRunnerRegistry() { + assertThat(jobManagerRunnerRegistry.getGlobalCleanupFuture()).isDone(); + + // the JobManagerRunnerRegistry needs to be cleaned up first + assertThat(jobGraphWriter.getGlobalCleanupFuture()).isNotDone(); + assertThat(blobServer.getGlobalCleanupFuture()).isNotDone(); + assertThat(highAvailabilityServices.getGlobalCleanupFuture()).isNotDone(); + } + + private void assertJobManagerMetricGroupNotCleaned() { + assertThat(jobManagerMetricGroup.numRegisteredJobMetricGroups()).isEqualTo(1); + } + + private void assertLocalCleanupTriggered() { + assertThat(jobManagerRunnerRegistry.getLocalCleanupFuture()).isCompleted(); + assertThat(jobGraphWriter.getLocalCleanupFuture()).isCompleted(); + assertThat(blobServer.getLocalCleanupFuture()).isCompleted(); + assertThat(highAvailabilityServices.getLocalCleanupFuture()).isCompleted(); + } + + private void assertGlobalCleanupTriggered() { + assertThat(jobManagerRunnerRegistry.getGlobalCleanupFuture()).isCompleted(); + assertThat(jobGraphWriter.getGlobalCleanupFuture()).isCompleted(); + assertThat(blobServer.getGlobalCleanupFuture()).isCompleted(); + assertThat(highAvailabilityServices.getGlobalCleanupFuture()).isCompleted(); + } + + private void assertJobManagerMetricGroupCleaned() { + assertThat(jobManagerMetricGroup.numRegisteredJobMetricGroups()).isEqualTo(0); + } + + private static class AbstractTestingCleanableResource + implements LocallyCleanableResource, GloballyCleanableResource { + + private final CompletableFuture localCleanupFuture = new CompletableFuture<>(); + private final CompletableFuture globalCleanupFuture = new CompletableFuture<>(); + + @Override + public void globalCleanup(JobID jobId) { + throw new UnsupportedOperationException("Synchronous globalCleanup is not supported."); + } + + @Override + public void localCleanup(JobID jobId) { + throw new UnsupportedOperationException("Synchronous localCleanup is not supported."); + } + + @Override + public CompletableFuture localCleanupAsync(JobID jobId, Executor ignoredExecutor) { + localCleanupFuture.complete(jobId); + + return FutureUtils.completedVoidFuture(); + } + + @Override + public CompletableFuture globalCleanupAsync(JobID jobId, Executor ignoredExecutor) { + globalCleanupFuture.complete(jobId); + + return FutureUtils.completedVoidFuture(); + } + + public CompletableFuture getLocalCleanupFuture() { + return localCleanupFuture; + } + + public CompletableFuture getGlobalCleanupFuture() { + return globalCleanupFuture; + } + } + + private static class CleanableJobGraphWriter extends AbstractTestingCleanableResource + implements JobGraphWriter { + + @Override + public void putJobGraph(JobGraph jobGraph) { + throw new UnsupportedOperationException("putJobGraph operation not supported."); + } + } + + private static class CleanableHighAvailabilityServices extends AbstractTestingCleanableResource + implements HighAvailabilityServices { + + @Override + public LeaderRetrievalService getResourceManagerLeaderRetriever() { + throw new UnsupportedOperationException( + "getResourceManagerLeaderRetriever operation not supported."); + } + + @Override + public LeaderRetrievalService getDispatcherLeaderRetriever() { + throw new UnsupportedOperationException( + "getDispatcherLeaderRetriever operation not supported."); + } + + @Override + public LeaderRetrievalService getJobManagerLeaderRetriever(JobID jobID) { + throw new UnsupportedOperationException( + "getJobManagerLeaderRetriever operation not supported."); + } + + @Override + public LeaderRetrievalService getJobManagerLeaderRetriever( + JobID jobID, String defaultJobManagerAddress) { + throw new UnsupportedOperationException( + "getJobManagerLeaderRetriever operation not supported."); + } + + @Override + public LeaderElectionService getResourceManagerLeaderElectionService() { + throw new UnsupportedOperationException( + "getResourceManagerLeaderElectionService operation not supported."); + } + + @Override + public LeaderElectionService getDispatcherLeaderElectionService() { + throw new UnsupportedOperationException( + "getDispatcherLeaderElectionService operation not supported."); + } + + @Override + public LeaderElectionService getJobManagerLeaderElectionService(JobID jobID) { + throw new UnsupportedOperationException( + "getJobManagerLeaderElectionService operation not supported."); + } + + @Override + public CheckpointRecoveryFactory getCheckpointRecoveryFactory() throws Exception { + throw new UnsupportedOperationException( + "getCheckpointRecoveryFactory operation not supported."); + } + + @Override + public JobGraphStore getJobGraphStore() throws Exception { + throw new UnsupportedOperationException("getJobGraphStore operation not supported."); + } + + @Override + public JobResultStore getJobResultStore() throws Exception { + throw new UnsupportedOperationException("getJobResultStore operation not supported."); + } + + @Override + public BlobStore createBlobStore() throws IOException { + throw new UnsupportedOperationException("createBlobStore operation not supported."); + } + + @Override + public void close() throws Exception { + throw new UnsupportedOperationException("close operation not supported."); + } + + @Override + public void closeAndCleanupAllData() throws Exception { + throw new UnsupportedOperationException( + "closeAndCleanupAllData operation not supported."); + } + } + + private static class CleanableBlobServer extends BlobServer { + + private final CompletableFuture localCleanupFuture = new CompletableFuture<>(); + private final CompletableFuture globalCleanupFuture = new CompletableFuture<>(); + + public CleanableBlobServer() throws IOException { + super( + new Configuration(), + new File("non-existent-file"), + new TestingBlobStoreBuilder().createTestingBlobStore()); + } + + @Override + public void globalCleanup(JobID jobId) { + throw new UnsupportedOperationException("Synchronous globalCleanup is not supported."); + } + + @Override + public void localCleanup(JobID jobId) { + throw new UnsupportedOperationException("Synchronous localCleanup is not supported."); + } + + @Override + public CompletableFuture localCleanupAsync(JobID jobId, Executor ignoredExecutor) { + localCleanupFuture.complete(jobId); + + return FutureUtils.completedVoidFuture(); + } + + @Override + public CompletableFuture globalCleanupAsync(JobID jobId, Executor ignoredExecutor) { + globalCleanupFuture.complete(jobId); + + return FutureUtils.completedVoidFuture(); + } + + public CompletableFuture getLocalCleanupFuture() { + return localCleanupFuture; + } + + public CompletableFuture getGlobalCleanupFuture() { + return globalCleanupFuture; + } + } + + private static class CleanableJobManagerRegistry extends JobManagerRunnerRegistry { + + private final CompletableFuture localCleanupFuture = new CompletableFuture<>(); + private final CompletableFuture globalCleanupFuture = new CompletableFuture<>(); + + private final CompletableFuture localCleanupResultFuture = new CompletableFuture<>(); + private final CompletableFuture globalCleanupResultFuture = new CompletableFuture<>(); + + public CleanableJobManagerRegistry() { + super(1); + } + + @Override + public void globalCleanup(JobID jobId) { + throw new UnsupportedOperationException("Synchronous globalCleanup is not supported."); + } + + @Override + public void localCleanup(JobID jobId) { + throw new UnsupportedOperationException("Synchronous localCleanup is not supported."); + } + + @Override + public CompletableFuture localCleanupAsync(JobID jobId, Executor ignoredExecutor) { + localCleanupFuture.complete(jobId); + + return localCleanupResultFuture; + } + + @Override + public CompletableFuture globalCleanupAsync(JobID jobId, Executor ignoredExecutor) { + globalCleanupFuture.complete(jobId); + + return globalCleanupResultFuture; + } + + public CompletableFuture getLocalCleanupFuture() { + return localCleanupFuture; + } + + public CompletableFuture getGlobalCleanupFuture() { + return globalCleanupFuture; + } + + public void completeLocalCleanup() { + localCleanupResultFuture.complete(null); + } + + public void completeGlobalCleanup() { + globalCleanupResultFuture.complete(null); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/GloballyCleanableResourceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/GloballyCleanableResourceTest.java new file mode 100644 index 0000000000000..00fe5641c4500 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/GloballyCleanableResourceTest.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.util.concurrent.Executors; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.CompletableFuture; + +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +/** + * {@code GloballyCleanableResourceTest} tests the default implementation of {@link + * GloballyCleanableResource}. + */ +public class GloballyCleanableResourceTest { + + @Test + public void testGlobalCleanupAsync() { + final CompletableFuture globalCleanupTriggered = new CompletableFuture<>(); + final GloballyCleanableResource testInstance = + TestingCleanableResource.builder() + .withGlobalCleanupConsumer(globalCleanupTriggered::complete) + .build(); + + final JobID jobId = new JobID(); + testInstance.globalCleanupAsync(jobId, Executors.directExecutor()); + + assertThat(globalCleanupTriggered).isCompletedWithValue(jobId); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/LocallyCleanableResourceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/LocallyCleanableResourceTest.java new file mode 100644 index 0000000000000..8db2841fccfdc --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/LocallyCleanableResourceTest.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.util.concurrent.Executors; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.CompletableFuture; + +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +/** + * {@code LocallyCleanableResourceTest} tests the default implementation of {@link + * LocallyCleanableResource}. + */ +public class LocallyCleanableResourceTest { + + @Test + public void testLocalCleanupAsync() { + final CompletableFuture localCleanupTriggered = new CompletableFuture<>(); + final LocallyCleanableResource testInstance = + TestingCleanableResource.builder() + .withLocalCleanupConsumer(localCleanupTriggered::complete) + .build(); + + final JobID jobId = new JobID(); + testInstance.localCleanupAsync(jobId, Executors.directExecutor()); + + assertThat(localCleanupTriggered).isCompletedWithValue(jobId); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingCleanableResource.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingCleanableResource.java new file mode 100644 index 0000000000000..0380e11c4208d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingCleanableResource.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.util.function.ThrowingConsumer; + +/** + * {@code TestingCleanableResource} provides methods for testing the callbacks of {@link + * LocallyCleanableResource}. + */ +class TestingCleanableResource implements LocallyCleanableResource, GloballyCleanableResource { + + private final ThrowingConsumer localCleanupConsumer; + private final ThrowingConsumer globalCleanupConsumer; + + private TestingCleanableResource( + ThrowingConsumer localCleanupConsumer, + ThrowingConsumer globalCleanupConsumer) { + this.localCleanupConsumer = localCleanupConsumer; + this.globalCleanupConsumer = globalCleanupConsumer; + } + + @Override + public void localCleanup(JobID jobId) throws Exception { + localCleanupConsumer.accept(jobId); + } + + @Override + public void globalCleanup(JobID jobId) throws Exception { + globalCleanupConsumer.accept(jobId); + } + + public static TestingCleanableResource.Builder builder() { + return new Builder(); + } + + static class Builder { + + private ThrowingConsumer localCleanupConsumer = + jobId -> { + throw new UnsupportedOperationException("Local cleanup is not supported."); + }; + private ThrowingConsumer globalCleanupConsumer = + jobId -> { + throw new UnsupportedOperationException("Global cleanup is not supported."); + }; + + private Builder() {} + + public Builder withLocalCleanupConsumer( + ThrowingConsumer localCleanupConsumer) { + this.localCleanupConsumer = localCleanupConsumer; + return this; + } + + public Builder withGlobalCleanupConsumer( + ThrowingConsumer globalCleanupConsumer) { + this.globalCleanupConsumer = globalCleanupConsumer; + return this; + } + + public TestingCleanableResource build() { + return new TestingCleanableResource(localCleanupConsumer, globalCleanupConsumer); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingCleanupRunnerFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingCleanupRunnerFactory.java new file mode 100644 index 0000000000000..223cc656abbec --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingCleanupRunnerFactory.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.dispatcher.TestingJobManagerRunnerFactory; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; + +import java.util.concurrent.Executor; + +/** + * {@code TestingCleanupRunnerFactory} implements {@link CleanupRunnerFactory} providing a factory + * method usually used for {@link CheckpointResourcesCleanupRunner} creations. + */ +public class TestingCleanupRunnerFactory extends TestingJobManagerRunnerFactory + implements CleanupRunnerFactory { + + public TestingCleanupRunnerFactory() { + super(0); + } + + @Override + public TestingJobManagerRunner create( + JobResult jobResult, + CheckpointRecoveryFactory checkpointRecoveryFactory, + Configuration configuration, + Executor cleanupExecutor) { + try { + return offerTestingJobManagerRunner(jobResult.getJobId()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingResourceCleanerFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingResourceCleanerFactory.java new file mode 100644 index 0000000000000..9d648dccfe46d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingResourceCleanerFactory.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.cleanup; + +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.concurrent.FutureUtils; + +import java.util.ArrayList; +import java.util.Collection; + +/** {@code TestingResourceCleanerFactory} for adding custom {@link ResourceCleaner} creation. */ +public class TestingResourceCleanerFactory implements ResourceCleanerFactory { + + private final Collection locallyCleanableResources = + new ArrayList<>(); + private final Collection globallyCleanableResources = + new ArrayList<>(); + + public + TestingResourceCleanerFactory with(T instance) { + return withLocallyCleanableResource(instance).withGloballyCleanableResource(instance); + } + + public TestingResourceCleanerFactory withLocallyCleanableResource( + LocallyCleanableResource locallyCleanableResource) { + this.locallyCleanableResources.add(locallyCleanableResource); + + return this; + } + + public TestingResourceCleanerFactory withGloballyCleanableResource( + GloballyCleanableResource globallyCleanableResource) { + this.globallyCleanableResources.add(globallyCleanableResource); + + return this; + } + + @Override + public ResourceCleaner createLocalResourceCleaner() { + return jobId -> { + Throwable t = null; + for (LocallyCleanableResource locallyCleanableResource : locallyCleanableResources) { + try { + locallyCleanableResource.localCleanup(jobId); + } catch (Throwable throwable) { + t = ExceptionUtils.firstOrSuppressed(throwable, t); + } + } + + return t != null + ? FutureUtils.completedExceptionally(t) + : FutureUtils.completedVoidFuture(); + }; + } + + @Override + public ResourceCleaner createGlobalResourceCleaner() { + return jobId -> { + Throwable t = null; + for (GloballyCleanableResource globallyCleanableResource : globallyCleanableResources) { + try { + globallyCleanableResource.globalCleanup(jobId); + } catch (Throwable throwable) { + t = ExceptionUtils.firstOrSuppressed(throwable, t); + } + } + + return t != null + ? FutureUtils.completedExceptionally(t) + : FutureUtils.completedVoidFuture(); + }; + } +} 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 03bf6dff8c3ee..ff1b2b7f52a6e 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 @@ -26,25 +26,26 @@ import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.DispatcherGateway; import org.apache.flink.runtime.dispatcher.DispatcherId; -import org.apache.flink.runtime.dispatcher.DispatcherOperationCaches; import org.apache.flink.runtime.dispatcher.DispatcherServices; import org.apache.flink.runtime.dispatcher.JobManagerRunnerFactory; -import org.apache.flink.runtime.dispatcher.MemoryExecutionGraphInfoStore; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobGraphStore; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobPersistenceComponents; import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; import org.apache.flink.runtime.dispatcher.SingleJobJobGraphStore; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; -import org.apache.flink.runtime.dispatcher.TestingJobManagerRunnerFactory; -import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist; -import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; -import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; +import org.apache.flink.runtime.dispatcher.TestingJobMasterServiceLeadershipRunnerFactory; +import org.apache.flink.runtime.dispatcher.TestingPartialDispatcherServices; +import org.apache.flink.runtime.dispatcher.cleanup.CleanupRunnerFactory; +import org.apache.flink.runtime.dispatcher.cleanup.TestingCleanupRunnerFactory; +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmanager.TestingJobPersistenceComponentFactory; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; -import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcServiceResource; import org.apache.flink.runtime.testutils.TestingJobGraphStore; @@ -66,7 +67,6 @@ import java.util.Collection; import java.util.UUID; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; import static org.hamcrest.Matchers.contains; @@ -95,6 +95,8 @@ public class DefaultDispatcherRunnerITCase extends TestLogger { private JobGraphStore jobGraphStore; + private JobResultStore jobResultStore; + private PartialDispatcherServices partialDispatcherServices; private DefaultDispatcherRunnerFactory dispatcherRunnerFactory; @@ -108,21 +110,12 @@ public void setup() { dispatcherLeaderElectionService = new TestingLeaderElectionService(); fatalErrorHandler = new TestingFatalErrorHandler(); jobGraphStore = TestingJobGraphStore.newBuilder().build(); + jobResultStore = new EmbeddedJobResultStore(); partialDispatcherServices = - new PartialDispatcherServices( - new Configuration(), - new TestingHighAvailabilityServicesBuilder().build(), - CompletableFuture::new, - blobServerResource.getBlobServer(), - new TestingHeartbeatServices(), - UnregisteredMetricGroups::createUnregisteredJobManagerMetricGroup, - new MemoryExecutionGraphInfoStore(), - fatalErrorHandler, - VoidHistoryServerArchivist.INSTANCE, - null, - ForkJoinPool.commonPool(), - new DispatcherOperationCaches()); + TestingPartialDispatcherServices.builder() + .withFatalErrorHandler(fatalErrorHandler) + .build(blobServerResource.getBlobServer(), new Configuration()); } @After @@ -176,11 +169,13 @@ private DispatcherGateway electLeaderAndRetrieveGateway(UUID firstLeaderSessionI @Test public void leaderChange_withBlockingJobManagerTermination_doesNotAffectNewLeader() throws Exception { - final TestingJobManagerRunnerFactory jobManagerRunnerFactory = - new TestingJobManagerRunnerFactory(1); + final TestingJobMasterServiceLeadershipRunnerFactory jobManagerRunnerFactory = + new TestingJobMasterServiceLeadershipRunnerFactory(1); + final TestingCleanupRunnerFactory cleanupRunnerFactory = new TestingCleanupRunnerFactory(); dispatcherRunnerFactory = DefaultDispatcherRunnerFactory.createSessionRunner( - new TestingDispatcherFactory(jobManagerRunnerFactory)); + new TestingDispatcherFactory( + jobManagerRunnerFactory, cleanupRunnerFactory)); jobGraphStore = new SingleJobJobGraphStore(jobGraph); try (final DispatcherRunner dispatcherRunner = createDispatcherRunner()) { @@ -230,9 +225,13 @@ public void leaderChange_withBlockingJobManagerTermination_doesNotAffectNewLeade private static class TestingDispatcherFactory implements DispatcherFactory { private final JobManagerRunnerFactory jobManagerRunnerFactory; + private final CleanupRunnerFactory cleanupRunnerFactory; - private TestingDispatcherFactory(JobManagerRunnerFactory jobManagerRunnerFactory) { + private TestingDispatcherFactory( + JobManagerRunnerFactory jobManagerRunnerFactory, + CleanupRunnerFactory cleanupRunnerFactory) { this.jobManagerRunnerFactory = jobManagerRunnerFactory; + this.cleanupRunnerFactory = cleanupRunnerFactory; } @Override @@ -240,17 +239,21 @@ public Dispatcher createDispatcher( RpcService rpcService, DispatcherId fencingToken, Collection recoveredJobs, + Collection recoveredDirtyJobResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, - PartialDispatcherServicesWithJobGraphStore - partialDispatcherServicesWithJobGraphStore) + PartialDispatcherServicesWithJobPersistenceComponents + partialDispatcherServicesWithJobPersistenceComponents) throws Exception { return new StandaloneDispatcher( rpcService, fencingToken, recoveredJobs, + recoveredDirtyJobResults, dispatcherBootstrapFactory, DispatcherServices.from( - partialDispatcherServicesWithJobGraphStore, jobManagerRunnerFactory)); + partialDispatcherServicesWithJobPersistenceComponents, + jobManagerRunnerFactory, + cleanupRunnerFactory)); } } @@ -262,7 +265,7 @@ private DispatcherRunner createDispatcherRunner() throws Exception { return dispatcherRunnerFactory.createDispatcherRunner( dispatcherLeaderElectionService, fatalErrorHandler, - () -> jobGraphStore, + new TestingJobPersistenceComponentFactory(jobGraphStore, jobResultStore), TestingUtils.defaultExecutor(), rpcServiceResource.getTestingRpcService(), partialDispatcherServices); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactoryTest.java new file mode 100644 index 0000000000000..257294e8c4b9f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/JobDispatcherLeaderProcessFactoryFactoryTest.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.dispatcher.runner; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.dispatcher.TestingPartialDispatcherServices; +import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; +import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore; +import org.apache.flink.runtime.jobmanager.TestingJobPersistenceComponentFactory; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler; +import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.testutils.TestingJobResultStore; +import org.apache.flink.util.CollectionUtil; +import org.apache.flink.util.TestLoggerExtension; +import org.apache.flink.util.concurrent.Executors; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.nio.file.Path; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +@ExtendWith(TestLoggerExtension.class) +class JobDispatcherLeaderProcessFactoryFactoryTest { + + @TempDir private Path temporaryFolder; + + @Test + public void testJobGraphWithoutDirtyJobResult() throws IOException { + final JobGraph jobGraph = JobGraphTestUtils.emptyJobGraph(); + + final JobDispatcherLeaderProcessFactory factory = + createDispatcherLeaderProcessFactoryFromTestInstance( + jobGraph, null, temporaryFolder); + + assertThat(factory.getJobGraph()).isEqualTo(jobGraph); + assertThat(factory.getRecoveredDirtyJobResult()).isNull(); + } + + @Test + public void testJobGraphWithMatchingDirtyJobResult() throws IOException { + final JobGraph jobGraph = JobGraphTestUtils.emptyJobGraph(); + final JobResult jobResult = + TestingJobResultStore.createSuccessfulJobResult(jobGraph.getJobID()); + + final JobDispatcherLeaderProcessFactory factory = + createDispatcherLeaderProcessFactoryFromTestInstance( + jobGraph, jobResult, temporaryFolder); + + assertThat(factory.getJobGraph()).isNull(); + assertThat(factory.getRecoveredDirtyJobResult()).isEqualTo(jobResult); + } + + @Test + public void testJobGraphWithNotMatchingDirtyJobResult() throws IOException { + final JobGraph jobGraph = JobGraphTestUtils.emptyJobGraph(); + final JobResult jobResult = TestingJobResultStore.createSuccessfulJobResult(new JobID()); + + final JobDispatcherLeaderProcessFactory factory = + createDispatcherLeaderProcessFactoryFromTestInstance( + jobGraph, jobResult, temporaryFolder); + + assertThat(factory.getJobGraph()).isEqualTo(jobGraph); + assertThat(factory.getRecoveredDirtyJobResult()).isNull(); + } + + @Test + public void testMissingJobGraph() throws IOException { + assertThatThrownBy( + () -> + createDispatcherLeaderProcessFactoryFromTestInstance( + null, + TestingJobResultStore.createSuccessfulJobResult( + new JobID()), + temporaryFolder)) + .isInstanceOf(NullPointerException.class); + } + + private static JobDispatcherLeaderProcessFactory + createDispatcherLeaderProcessFactoryFromTestInstance( + @Nullable JobGraph jobGraph, + @Nullable JobResult dirtyJobResult, + Path storageDir) + throws IOException { + final JobDispatcherLeaderProcessFactoryFactory testInstance = + new JobDispatcherLeaderProcessFactoryFactory(ignoredConfig -> jobGraph); + + final TestingJobResultStore jobResultStore = + TestingJobResultStore.builder() + .withGetDirtyResultsSupplier( + () -> CollectionUtil.ofNullable(dirtyJobResult)) + .build(); + final JobGraphStore jobGraphStore = new StandaloneJobGraphStore(); + return testInstance.createFactory( + new TestingJobPersistenceComponentFactory(jobGraphStore, jobResultStore), + Executors.directExecutor(), + new TestingRpcService(), + TestingPartialDispatcherServices.builder() + .withHighAvailabilityServices( + new TestingHighAvailabilityServicesBuilder() + .setJobGraphStore(jobGraphStore) + .setJobResultStore(jobResultStore) + .build()) + .build(storageDir.toFile(), new Configuration()), + NoOpFatalErrorHandler.INSTANCE); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java index 12cdcaa5d1918..7dd2446ce9f0a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java @@ -19,50 +19,55 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.api.common.JobID; +import org.apache.flink.core.testutils.FlinkAssertions; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.client.JobSubmissionException; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.jobmanager.JobGraphStore; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.testutils.TestingJobGraphStore; +import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FlinkException; -import org.apache.flink.util.TestLogger; +import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.function.ThrowingConsumer; -import org.apache.flink.util.function.TriFunctionWithException; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; -import java.time.Duration; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Supplier; -import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; -import static org.apache.flink.core.testutils.FlinkMatchers.willNotComplete; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.apache.flink.core.testutils.FlinkAssertions.STREAM_THROWABLE; +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the {@link SessionDispatcherLeaderProcess}. */ -public class SessionDispatcherLeaderProcessTest extends TestLogger { +@ExtendWith(TestLoggerExtension.class) +public class SessionDispatcherLeaderProcessTest { private static final JobGraph JOB_GRAPH = JobGraphTestUtils.emptyJobGraph(); @@ -73,22 +78,27 @@ public class SessionDispatcherLeaderProcessTest extends TestLogger { private TestingFatalErrorHandler fatalErrorHandler; private JobGraphStore jobGraphStore; + private JobResultStore jobResultStore; - private TestingDispatcherServiceFactory dispatcherServiceFactory; + private AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory + dispatcherServiceFactory; - @BeforeClass + @BeforeAll public static void setupClass() { ioExecutor = Executors.newSingleThreadExecutor(); } - @Before + @BeforeEach public void setup() { fatalErrorHandler = new TestingFatalErrorHandler(); jobGraphStore = TestingJobGraphStore.newBuilder().build(); - dispatcherServiceFactory = TestingDispatcherServiceFactory.newBuilder().build(); + jobResultStore = TestingJobResultStore.builder().build(); + dispatcherServiceFactory = + createFactoryBasedOnGenericSupplier( + () -> TestingDispatcherGatewayService.newBuilder().build()); } - @After + @AfterEach public void teardown() throws Exception { if (fatalErrorHandler != null) { fatalErrorHandler.rethrowError(); @@ -96,7 +106,7 @@ public void teardown() throws Exception { } } - @AfterClass + @AfterAll public static void teardownClass() { if (ioExecutor != null) { ExecutorUtils.gracefulShutdown(5L, TimeUnit.SECONDS, ioExecutor); @@ -111,39 +121,119 @@ public void start_afterClose_doesNotHaveAnEffect() throws Exception { dispatcherLeaderProcess.close(); dispatcherLeaderProcess.start(); - assertThat( - dispatcherLeaderProcess.getState(), - is(SessionDispatcherLeaderProcess.State.STOPPED)); + assertThat(dispatcherLeaderProcess.getState()) + .isEqualTo(SessionDispatcherLeaderProcess.State.STOPPED); + } + + @Test + public void testStartTriggeringDispatcherServiceCreation() throws Exception { + dispatcherServiceFactory = + createFactoryBasedOnGenericSupplier( + () -> TestingDispatcherGatewayService.newBuilder().build()); + + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = + createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + assertThat(dispatcherLeaderProcess.getState()) + .isEqualTo(SessionDispatcherLeaderProcess.State.RUNNING); + } + } + + @Test + public void testRecoveryWithJobGraphButNoDirtyJobResult() throws Exception { + testJobRecovery( + Collections.singleton(JOB_GRAPH), + Collections.emptySet(), + actualRecoveredJobGraphs -> + assertThat(actualRecoveredJobGraphs).singleElement().isEqualTo(JOB_GRAPH), + actualRecoveredDirtyJobResults -> + assertThat(actualRecoveredDirtyJobResults).isEmpty()); } @Test - public void start_triggersJobGraphRecoveryAndDispatcherServiceCreation() throws Exception { + public void testRecoveryWithJobGraphAndMatchingDirtyJobResult() throws Exception { + final JobResult matchingDirtyJobResult = + TestingJobResultStore.createSuccessfulJobResult(JOB_GRAPH.getJobID()); + + testJobRecovery( + Collections.singleton(JOB_GRAPH), + Collections.singleton(matchingDirtyJobResult), + actualRecoveredJobGraphs -> assertThat(actualRecoveredJobGraphs).isEmpty(), + actualRecoveredDirtyJobResults -> + assertThat(actualRecoveredDirtyJobResults) + .singleElement() + .isEqualTo(matchingDirtyJobResult)); + } + + @Test + public void testRecoveryWithMultipleJobGraphsAndOneMatchingDirtyJobResult() throws Exception { + final JobResult matchingDirtyJobResult = + TestingJobResultStore.createSuccessfulJobResult(JOB_GRAPH.getJobID()); + final JobGraph otherJobGraph = JobGraphTestUtils.emptyJobGraph(); + + testJobRecovery( + Arrays.asList(otherJobGraph, JOB_GRAPH), + Collections.singleton(matchingDirtyJobResult), + actualRecoveredJobGraphs -> + assertThat(actualRecoveredJobGraphs) + .singleElement() + .isEqualTo(otherJobGraph), + actualRecoveredDirtyJobResults -> + assertThat(actualRecoveredDirtyJobResults) + .singleElement() + .isEqualTo(matchingDirtyJobResult)); + } + + @Test + public void testRecoveryWithoutJobGraphButDirtyJobResult() throws Exception { + final JobResult dirtyJobResult = + TestingJobResultStore.createSuccessfulJobResult(new JobID()); + + testJobRecovery( + Collections.emptyList(), + Collections.singleton(dirtyJobResult), + actualRecoveredJobGraphs -> assertThat(actualRecoveredJobGraphs).isEmpty(), + actualRecoveredDirtyJobResults -> + assertThat(actualRecoveredDirtyJobResults) + .singleElement() + .isEqualTo(dirtyJobResult)); + } + + private void testJobRecovery( + Collection jobGraphsToRecover, + Set dirtyJobResults, + Consumer> recoveredJobGraphAssertion, + Consumer> recoveredDirtyJobResultAssertion) + throws Exception { jobGraphStore = - TestingJobGraphStore.newBuilder() - .setInitialJobGraphs(Collections.singleton(JOB_GRAPH)) + TestingJobGraphStore.newBuilder().setInitialJobGraphs(jobGraphsToRecover).build(); + + jobResultStore = + TestingJobResultStore.builder() + .withGetDirtyResultsSupplier(() -> dirtyJobResults) .build(); final CompletableFuture> recoveredJobGraphsFuture = new CompletableFuture<>(); + final CompletableFuture> recoveredDirtyJobResultsFuture = + new CompletableFuture<>(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (fencingToken, recoveredJobGraphs, jobGraphStore) -> { - recoveredJobGraphsFuture.complete(recoveredJobGraphs); - return TestingDispatcherGatewayService.newBuilder().build(); - }) - .build(); + (ignoredDispatcherId, + recoveredJobs, + recoveredDirtyJobResults, + ignoredJobGraphWriter, + ignoredJobResultStore) -> { + recoveredJobGraphsFuture.complete(recoveredJobs); + recoveredDirtyJobResultsFuture.complete(recoveredDirtyJobResults); + return TestingDispatcherGatewayService.newBuilder().build(); + }; try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); - assertThat( - dispatcherLeaderProcess.getState(), - is(SessionDispatcherLeaderProcess.State.RUNNING)); - - final Collection recoveredJobGraphs = recoveredJobGraphsFuture.get(); - assertThat(recoveredJobGraphs, containsInAnyOrder(JOB_GRAPH)); + recoveredJobGraphAssertion.accept(recoveredJobGraphsFuture.get()); + recoveredDirtyJobResultAssertion.accept(recoveredDirtyJobResultsFuture.get()); } } @@ -158,15 +248,12 @@ public void closeAsync_stopsJobGraphStoreAndDispatcher() throws Exception { final CompletableFuture dispatcherServiceTerminationFuture = new CompletableFuture<>(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (ignoredA, ignoredB, ignoredC) -> - TestingDispatcherGatewayService.newBuilder() - .setTerminationFuture( - dispatcherServiceTerminationFuture) - .withManualTerminationFutureCompletion() - .build()) - .build(); + createFactoryBasedOnGenericSupplier( + () -> + TestingDispatcherGatewayService.newBuilder() + .setTerminationFuture(dispatcherServiceTerminationFuture) + .withManualTerminationFutureCompletion() + .build()); try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { @@ -177,8 +264,8 @@ public void closeAsync_stopsJobGraphStoreAndDispatcher() throws Exception { final CompletableFuture terminationFuture = dispatcherLeaderProcess.closeAsync(); - assertThat(jobGraphStopFuture.isDone(), is(false)); - assertThat(terminationFuture.isDone(), is(false)); + assertThat(jobGraphStopFuture).isNotDone(); + assertThat(terminationFuture).isNotDone(); dispatcherServiceTerminationFuture.complete(null); @@ -194,13 +281,12 @@ public void closeAsync_stopsJobGraphStoreAndDispatcher() throws Exception { public void unexpectedDispatcherServiceTerminationWhileRunning_callsFatalErrorHandler() { final CompletableFuture terminationFuture = new CompletableFuture<>(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (ignoredA, ignoredB, ignoredC) -> - TestingDispatcherGatewayService.newBuilder() - .setTerminationFuture(terminationFuture) - .build()) - .build(); + createFactoryBasedOnGenericSupplier( + () -> + TestingDispatcherGatewayService.newBuilder() + .setTerminationFuture(terminationFuture) + .build()); + final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess(); dispatcherLeaderProcess.start(); @@ -209,7 +295,7 @@ public void unexpectedDispatcherServiceTerminationWhileRunning_callsFatalErrorHa terminationFuture.completeExceptionally(expectedFailure); final Throwable error = fatalErrorHandler.getErrorFuture().join(); - assertThat(error, containsCause(expectedFailure)); + assertThat(error).getRootCause().isEqualTo(expectedFailure); fatalErrorHandler.clearError(); } @@ -219,14 +305,12 @@ public void unexpectedDispatcherServiceTerminationWhileRunning_callsFatalErrorHa unexpectedDispatcherServiceTerminationWhileNotRunning_doesNotCallFatalErrorHandler() { final CompletableFuture terminationFuture = new CompletableFuture<>(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (ignoredA, ignoredB, ignoredC) -> - TestingDispatcherGatewayService.newBuilder() - .setTerminationFuture(terminationFuture) - .withManualTerminationFutureCompletion() - .build()) - .build(); + createFactoryBasedOnGenericSupplier( + () -> + TestingDispatcherGatewayService.newBuilder() + .setTerminationFuture(terminationFuture) + .withManualTerminationFutureCompletion() + .build()); final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess(); dispatcherLeaderProcess.start(); @@ -236,7 +320,8 @@ public void unexpectedDispatcherServiceTerminationWhileRunning_callsFatalErrorHa final FlinkException expectedFailure = new FlinkException("Expected test failure."); terminationFuture.completeExceptionally(expectedFailure); - assertThat(fatalErrorHandler.getErrorFuture(), willNotComplete(Duration.ofMillis(10))); + assertThatThrownBy(() -> fatalErrorHandler.getErrorFuture().get(10, TimeUnit.MILLISECONDS)) + .isInstanceOf(TimeoutException.class); } @Test @@ -248,16 +333,17 @@ public void confirmLeaderSessionFuture_completesAfterDispatcherServiceHasBeenSta TestingDispatcherGateway.newBuilder().setAddress(dispatcherAddress).build(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - TriFunctionWithException.unchecked( - (ignoredA, ignoredB, ignoredC) -> { - createDispatcherServiceLatch.await(); - return TestingDispatcherGatewayService.newBuilder() - .setDispatcherGateway(dispatcherGateway) - .build(); - })) - .build(); + createFactoryBasedOnGenericSupplier( + () -> { + try { + createDispatcherServiceLatch.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + return TestingDispatcherGatewayService.newBuilder() + .setDispatcherGateway(dispatcherGateway) + .build(); + }); try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { @@ -266,11 +352,13 @@ public void confirmLeaderSessionFuture_completesAfterDispatcherServiceHasBeenSta dispatcherLeaderProcess.start(); - assertThat(confirmLeaderSessionFuture.isDone(), is(false)); + assertThat(confirmLeaderSessionFuture).isNotDone(); createDispatcherServiceLatch.trigger(); - assertThat(confirmLeaderSessionFuture.get(), is(dispatcherAddress)); + assertThat(confirmLeaderSessionFuture) + .succeedsWithin(100, TimeUnit.MILLISECONDS) + .isEqualTo(dispatcherAddress); } } @@ -291,13 +379,11 @@ public void closeAsync_duringJobRecovery_preventsDispatcherServiceCreation() thr .build(); this.dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (ignoredA, ignoredB, ignoredC) -> { - createDispatcherServiceLatch.trigger(); - return TestingDispatcherGatewayService.newBuilder().build(); - }) - .build(); + createFactoryBasedOnGenericSupplier( + () -> { + createDispatcherServiceLatch.trigger(); + return TestingDispatcherGatewayService.newBuilder().build(); + }); try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { @@ -309,11 +395,10 @@ public void closeAsync_duringJobRecovery_preventsDispatcherServiceCreation() thr completeJobRecoveryLatch.trigger(); - try { - createDispatcherServiceLatch.await(10L, TimeUnit.MILLISECONDS); - fail("No dispatcher service should be created after the process has been stopped."); - } catch (TimeoutException expected) { - } + assertThatThrownBy( + () -> createDispatcherServiceLatch.await(10L, TimeUnit.MILLISECONDS), + "No dispatcher service should be created after the process has been stopped.") + .isInstanceOf(TimeoutException.class); } } @@ -335,11 +420,7 @@ public void onRemovedJobGraph_terminatesRunningJob() throws Exception { .build(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (dispatcherId, jobGraphs, jobGraphWriter) -> - testingDispatcherService) - .build(); + createFactoryBasedOnGenericSupplier(() -> testingDispatcherService); try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { @@ -349,10 +430,10 @@ public void onRemovedJobGraph_terminatesRunningJob() throws Exception { dispatcherLeaderProcess.getDispatcherGateway().get(); // now remove the Job from the JobGraphStore and notify the dispatcher service - jobGraphStore.removeJobGraph(JOB_GRAPH.getJobID()); + jobGraphStore.globalCleanup(JOB_GRAPH.getJobID()); dispatcherLeaderProcess.onRemovedJobGraph(JOB_GRAPH.getJobID()); - assertThat(terminateJobFuture.get(), is(JOB_GRAPH.getJobID())); + assertThat(terminateJobFuture).isCompletedWithValue(JOB_GRAPH.getJobID()); } } @@ -367,11 +448,7 @@ public void onRemovedJobGraph_failingRemovalCall_failsFatally() throws Exception .build(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (dispatcherId, jobGraphs, jobGraphWriter) -> - testingDispatcherService) - .build(); + createFactoryBasedOnGenericSupplier(() -> testingDispatcherService); try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { @@ -385,9 +462,7 @@ public void onRemovedJobGraph_failingRemovalCall_failsFatally() throws Exception final Throwable fatalError = fatalErrorHandler.getErrorFuture().join(); - assertTrue( - ExceptionUtils.findThrowable(fatalError, cause -> cause.equals(testException)) - .isPresent()); + assertThat(fatalError).hasCause(testException); fatalErrorHandler.clearError(); } @@ -405,7 +480,12 @@ public void onAddedJobGraph_submitsRecoveredJob() throws Exception { }) .build(); - dispatcherServiceFactory = createDispatcherServiceFactoryFor(testingDispatcherGateway); + dispatcherServiceFactory = + createFactoryBasedOnGenericSupplier( + () -> + TestingDispatcherGatewayService.newBuilder() + .setDispatcherGateway(testingDispatcherGateway) + .build()); try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { @@ -419,7 +499,7 @@ public void onAddedJobGraph_submitsRecoveredJob() throws Exception { final JobGraph submittedJobGraph = submittedJobFuture.get(); - assertThat(submittedJobGraph.getJobID(), is(JOB_GRAPH.getJobID())); + assertThat(submittedJobGraph.getJobID()).isEqualTo(JOB_GRAPH.getJobID()); } } @@ -449,11 +529,10 @@ public void onAddedJobGraph_ifNotRunning_isBeingIgnored() throws Exception { dispatcherLeaderProcess.onAddedJobGraph(JOB_GRAPH.getJobID()); - try { - recoveredJobFuture.get(10L, TimeUnit.MILLISECONDS); - fail("onAddedJobGraph should be ignored if the leader process is not running."); - } catch (TimeoutException expected) { - } + assertThatThrownBy( + () -> recoveredJobFuture.get(10L, TimeUnit.MILLISECONDS), + "onAddedJobGraph should be ignored if the leader process is not running.") + .isInstanceOf(TimeoutException.class); } } @@ -478,15 +557,13 @@ public void onAddedJobGraph_failingRecovery_propagatesTheFailure() throws Except jobGraphStore.putJobGraph(JOB_GRAPH); dispatcherLeaderProcess.onAddedJobGraph(JOB_GRAPH.getJobID()); - final CompletableFuture errorFuture = fatalErrorHandler.getErrorFuture(); - final Throwable throwable = errorFuture.get(); - assertThat( - ExceptionUtils.findThrowable(throwable, expectedFailure::equals).isPresent(), - is(true)); + assertThat(fatalErrorHandler.getErrorFuture()) + .succeedsWithin(100, TimeUnit.MILLISECONDS) + .extracting(FlinkAssertions::chainOfCauses, STREAM_THROWABLE) + .contains(expectedFailure); - assertThat( - dispatcherLeaderProcess.getState(), - is(SessionDispatcherLeaderProcess.State.STOPPED)); + assertThat(dispatcherLeaderProcess.getState()) + .isEqualTo(SessionDispatcherLeaderProcess.State.STOPPED); fatalErrorHandler.clearError(); } @@ -527,11 +604,15 @@ private void runJobRecoveryFailureTest(FlinkException testException) throws Exce dispatcherLeaderProcess.start(); // we expect that a fatal error occurred - final Throwable error = fatalErrorHandler.getErrorFuture().get(); - assertThat( - ExceptionUtils.findThrowableWithMessage(error, testException.getMessage()) - .isPresent(), - is(true)); + assertThat(fatalErrorHandler.getErrorFuture()) + .succeedsWithin(100, TimeUnit.MILLISECONDS) + .satisfies( + error -> + assertThat(error) + .satisfies( + anyCauseMatches( + testException.getClass(), + testException.getMessage()))); fatalErrorHandler.clearError(); } @@ -555,9 +636,9 @@ private void verifyOnAddedJobGraphResultFailsFatally( TestingFatalErrorHandler fatalErrorHandler) { final Throwable actualCause = fatalErrorHandler.getErrorFuture().join(); - assertTrue( - ExceptionUtils.findThrowable(actualCause, JobSubmissionException.class) - .isPresent()); + assertThat(actualCause) + .extracting(FlinkAssertions::chainOfCauses, FlinkAssertions.STREAM_THROWABLE) + .hasAtLeastOneElementOfType(JobSubmissionException.class); fatalErrorHandler.clearError(); } @@ -586,16 +667,14 @@ private void runOnAddedJobGraphTest( .setInitialJobGraphs(Collections.singleton(JOB_GRAPH)) .build(); dispatcherServiceFactory = - TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (dispatcherId, jobGraphs, jobGraphWriter) -> { - assertThat(jobGraphs, containsInAnyOrder(JOB_GRAPH)); - - return TestingDispatcherGatewayService.newBuilder() - .setDispatcherGateway(dispatcherGateway) - .build(); - }) - .build(); + createFactoryBasedOnJobGraphs( + jobGraphs -> { + assertThat(jobGraphs).containsExactlyInAnyOrder(JOB_GRAPH); + + return TestingDispatcherGatewayService.newBuilder() + .setDispatcherGateway(dispatcherGateway) + .build(); + }); try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { @@ -609,25 +688,34 @@ private void runOnAddedJobGraphTest( } } - private void verifyOnAddedJobGraphResultDidNotFail(TestingFatalErrorHandler fatalErrorHandler) - throws Exception { - try { - fatalErrorHandler.getErrorFuture().get(10L, TimeUnit.MILLISECONDS); - fail( - "Expected that duplicate job submissions due to false job recoveries are ignored."); - } catch (TimeoutException expected) { - } + private AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory + createFactoryBasedOnJobGraphs( + Function< + Collection, + AbstractDispatcherLeaderProcess.DispatcherGatewayService> + createFunction) { + return (ignoredDispatcherId, + recoveredJobs, + ignoredRecoveredDirtyJobResults, + ignoredJobGraphWriter, + ignoredJobResultStore) -> createFunction.apply(recoveredJobs); } - private TestingDispatcherServiceFactory createDispatcherServiceFactoryFor( - TestingDispatcherGateway testingDispatcherGateway) { - return TestingDispatcherServiceFactory.newBuilder() - .setCreateFunction( - (ignoredA, ignoredB, ignoredC) -> - TestingDispatcherGatewayService.newBuilder() - .setDispatcherGateway(testingDispatcherGateway) - .build()) - .build(); + private AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory + createFactoryBasedOnGenericSupplier( + Supplier supplier) { + return (ignoredDispatcherId, + ignoredRecoveredJobs, + ignoredRecoveredDirtyJobResults, + ignoredJobGraphWriter, + ignoredJobResultStore) -> supplier.get(); + } + + private void verifyOnAddedJobGraphResultDidNotFail(TestingFatalErrorHandler fatalErrorHandler) { + assertThatThrownBy( + () -> fatalErrorHandler.getErrorFuture().get(10L, TimeUnit.MILLISECONDS), + "Expected that duplicate job submissions due to false job recoveries are ignored.") + .isInstanceOf(TimeoutException.class); } private SessionDispatcherLeaderProcess createDispatcherLeaderProcess() { @@ -635,6 +723,7 @@ private SessionDispatcherLeaderProcess createDispatcherLeaderProcess() { leaderSessionId, dispatcherServiceFactory, jobGraphStore, + jobResultStore, ioExecutor, fatalErrorHandler); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java deleted file mode 100644 index 5efacdafeab5a..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/TestingDispatcherServiceFactory.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.dispatcher.runner; - -import org.apache.flink.runtime.dispatcher.DispatcherId; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobmanager.JobGraphWriter; -import org.apache.flink.util.function.TriFunction; - -import java.util.Collection; - -class TestingDispatcherServiceFactory - implements AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory { - private final TriFunction< - DispatcherId, - Collection, - JobGraphWriter, - AbstractDispatcherLeaderProcess.DispatcherGatewayService> - createFunction; - - private TestingDispatcherServiceFactory( - TriFunction< - DispatcherId, - Collection, - JobGraphWriter, - AbstractDispatcherLeaderProcess.DispatcherGatewayService> - createFunction) { - this.createFunction = createFunction; - } - - @Override - public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( - DispatcherId fencingToken, - Collection recoveredJobs, - JobGraphWriter jobGraphWriter) { - return createFunction.apply(fencingToken, recoveredJobs, jobGraphWriter); - } - - public static Builder newBuilder() { - return new Builder(); - } - - public static class Builder { - private TriFunction< - DispatcherId, - Collection, - JobGraphWriter, - AbstractDispatcherLeaderProcess.DispatcherGatewayService> - createFunction = - (ignoredA, ignoredB, ignoredC) -> - TestingDispatcherGatewayService.newBuilder().build(); - - private Builder() {} - - Builder setCreateFunction( - TriFunction< - DispatcherId, - Collection, - JobGraphWriter, - AbstractDispatcherLeaderProcess.DispatcherGatewayService> - createFunction) { - this.createFunction = createFunction; - return this; - } - - public TestingDispatcherServiceFactory build() { - return new TestingDispatcherServiceFactory(createFunction); - } - } -} 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 6f0322145e961..ca45d4e90aa1c 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 @@ -35,14 +35,15 @@ import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist; import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; +import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; -import org.apache.flink.runtime.highavailability.zookeeper.ZooKeeperRunningJobsRegistry; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobmanager.JobGraphStore; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; @@ -151,8 +152,6 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception { .asCuratorFramework(); try (final TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServicesBuilder() - .setRunningJobsRegistry( - new ZooKeeperRunningJobsRegistry(client, configuration)) .setDispatcherLeaderElectionService(dispatcherLeaderElectionService) .setJobMasterLeaderRetrieverFunction( jobId -> ZooKeeperUtils.createLeaderRetrievalService(client)) @@ -181,7 +180,17 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception { createDispatcherRunner( rpcService, dispatcherLeaderElectionService, - () -> createZooKeeperJobGraphStore(client), + new JobPersistenceComponentFactory() { + @Override + public JobGraphStore createJobGraphStore() { + return createZooKeeperJobGraphStore(client); + } + + @Override + public JobResultStore createJobResultStore() { + return new EmbeddedJobResultStore(); + } + }, partialDispatcherServices, defaultDispatcherRunnerFactory)) { @@ -229,14 +238,14 @@ public void testResourceCleanupUnderLeadershipChange() throws Exception { private DispatcherRunner createDispatcherRunner( TestingRpcService rpcService, TestingLeaderElectionService dispatcherLeaderElectionService, - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, PartialDispatcherServices partialDispatcherServices, DispatcherRunnerFactory dispatcherRunnerFactory) throws Exception { return dispatcherRunnerFactory.createDispatcherRunner( dispatcherLeaderElectionService, fatalErrorHandler, - jobGraphStoreFactory, + jobPersistenceComponentFactory, TestingUtils.defaultExecutor(), rpcService, partialDispatcherServices); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/ClusterEntrypointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/ClusterEntrypointTest.java index 160119bb8ed20..5b3926996df6c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/ClusterEntrypointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/entrypoint/ClusterEntrypointTest.java @@ -35,7 +35,7 @@ import org.apache.flink.runtime.entrypoint.component.DispatcherResourceManagerComponentFactory; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServicesBuilder; -import org.apache.flink.runtime.jobmanager.JobGraphStoreFactory; +import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; @@ -418,7 +418,7 @@ private TestingDispatcherRunnerFactory( public DispatcherRunner createDispatcherRunner( LeaderElectionService leaderElectionService, FatalErrorHandler fatalErrorHandler, - JobGraphStoreFactory jobGraphStoreFactory, + JobPersistenceComponentFactory jobPersistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java index b2e1b3d6d4b66..f849d65b37435 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/execution/librarycache/BlobLibraryCacheRecoveryITCase.java @@ -160,7 +160,7 @@ public void testRecoveryRegisterAndDownload() throws Exception { } // Remove blobs again - server[1].cleanupJob(jobId, true); + server[1].globalCleanup(jobId); // Verify everything is clean below recoveryDir/ final String clusterId = config.getString(HighAvailabilityOptions.HA_CLUSTER_ID); 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 e751f64ad9d6b..a79592795e24e 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 @@ -152,7 +152,7 @@ public void testSerialization() throws IOException, ClassNotFoundException { @Test public void testCreateFromInitializingJobForSuspendedJob() { final ArchivedExecutionGraph suspendedExecutionGraph = - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( new JobID(), "TestJob", JobStatus.SUSPENDED, @@ -170,7 +170,7 @@ public void testCheckpointSettingsArchiving() { CheckpointCoordinatorConfiguration.builder().build(); final ArchivedExecutionGraph archivedGraph = - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( new JobID(), "TestJob", JobStatus.INITIALIZING, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHaServicesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHaServicesTest.java index 630ba1f326487..433ed3aa7cb55 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHaServicesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHaServicesTest.java @@ -26,10 +26,12 @@ import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; +import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.Executors; import org.apache.flink.util.function.RunnableWithException; +import org.apache.flink.util.function.ThrowingConsumer; import org.junit.Test; @@ -39,7 +41,6 @@ import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; -import java.util.function.Consumer; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.is; @@ -129,7 +130,7 @@ public void testCleanupJobData() throws Exception { () -> {}, jobCleanupFuture::complete); - haServices.cleanupJobData(jobID); + haServices.globalCleanup(jobID); JobID jobIDCleaned = jobCleanupFuture.get(); assertThat(jobIDCleaned, is(jobID)); } @@ -184,7 +185,7 @@ private static final class TestingHaServices extends AbstractHaServices { private final Queue closeOperations; private final RunnableWithException internalCleanupRunnable; - private final Consumer internalJobCleanupConsumer; + private final ThrowingConsumer internalJobCleanupConsumer; private TestingHaServices( Configuration config, @@ -192,8 +193,18 @@ private TestingHaServices( BlobStoreService blobStoreService, Queue closeOperations, RunnableWithException internalCleanupRunnable, - Consumer internalJobCleanupConsumer) { - super(config, ioExecutor, blobStoreService); + ThrowingConsumer internalJobCleanupConsumer) { + super( + config, + ioExecutor, + blobStoreService, + TestingJobResultStore.builder() + .withMarkResultAsCleanConsumer( + ignoredJobId -> { + throw new AssertionError( + "Marking the job as clean shouldn't happen in the HaServices cleanup"); + }) + .build()); this.closeOperations = closeOperations; this.internalCleanupRunnable = internalCleanupRunnable; this.internalJobCleanupConsumer = internalJobCleanupConsumer; @@ -219,11 +230,6 @@ protected JobGraphStore createJobGraphStore() throws Exception { throw new UnsupportedOperationException("Not supported by this test implementation."); } - @Override - protected RunningJobsRegistry createRunningJobsRegistry() { - throw new UnsupportedOperationException("Not supported by this test implementation."); - } - @Override protected void internalClose() { closeOperations.offer(CloseOperations.HA_CLOSE); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java index 4434eefc852be..e4c573f21f4cf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java @@ -22,7 +22,7 @@ import org.apache.flink.runtime.blob.BlobStore; import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; -import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -67,13 +67,14 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices private volatile JobGraphStore jobGraphStore; - private volatile RunningJobsRegistry runningJobsRegistry = new StandaloneRunningJobsRegistry(); + private volatile JobResultStore jobResultStore = new EmbeddedJobResultStore(); private CompletableFuture closeFuture = new CompletableFuture<>(); private CompletableFuture closeAndCleanupAllDataFuture = new CompletableFuture<>(); - private volatile CompletableFuture jobCleanupFuture; + private volatile CompletableFuture globalCleanupFuture; + private volatile CompletableFuture localCleanupFuture; // ------------------------------------------------------------------------ // Setters for mock / testing implementations @@ -125,8 +126,8 @@ public void setJobGraphStore(JobGraphStore jobGraphStore) { this.jobGraphStore = jobGraphStore; } - public void setRunningJobsRegistry(RunningJobsRegistry runningJobsRegistry) { - this.runningJobsRegistry = runningJobsRegistry; + public void setJobResultStore(JobResultStore jobResultStore) { + this.jobResultStore = jobResultStore; } public void setJobMasterLeaderElectionServiceFunction( @@ -148,8 +149,12 @@ public void setCloseAndCleanupAllDataFuture( this.closeAndCleanupAllDataFuture = closeAndCleanupAllDataFuture; } - public void setCleanupJobDataFuture(CompletableFuture jobCleanupFuture) { - this.jobCleanupFuture = jobCleanupFuture; + public void setGlobalCleanupFuture(CompletableFuture globalCleanupFuture) { + this.globalCleanupFuture = globalCleanupFuture; + } + + public void setLocalCleanupFuture(CompletableFuture localCleanupFuture) { + this.localCleanupFuture = localCleanupFuture; } // ------------------------------------------------------------------------ @@ -262,8 +267,8 @@ public JobGraphStore getJobGraphStore() { } @Override - public RunningJobsRegistry getRunningJobsRegistry() { - return runningJobsRegistry; + public JobResultStore getJobResultStore() { + return jobResultStore; } @Override @@ -286,7 +291,12 @@ public void closeAndCleanupAllData() throws Exception { } @Override - public void cleanupJobData(JobID jobID) { - Optional.ofNullable(jobCleanupFuture).ifPresent(f -> f.complete(jobID)); + public void globalCleanup(JobID jobID) throws Exception { + Optional.ofNullable(globalCleanupFuture).ifPresent(f -> f.complete(jobID)); + } + + @Override + public void localCleanup(JobID jobID) throws Exception { + Optional.ofNullable(localCleanupFuture).ifPresent(f -> f.complete(jobID)); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServicesBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServicesBuilder.java index dbf52699c54fd..88557c02c85d5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServicesBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServicesBuilder.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; -import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore; import org.apache.flink.runtime.leaderelection.LeaderElectionService; @@ -69,7 +69,7 @@ public class TestingHighAvailabilityServicesBuilder { private JobGraphStore jobGraphStore = new StandaloneJobGraphStore(); - private RunningJobsRegistry runningJobsRegistry = new StandaloneRunningJobsRegistry(); + private JobResultStore jobResultStore = new EmbeddedJobResultStore(); private CompletableFuture closeFuture = new CompletableFuture<>(); @@ -99,7 +99,7 @@ public TestingHighAvailabilityServices build() { testingHighAvailabilityServices.setCheckpointRecoveryFactory(checkpointRecoveryFactory); testingHighAvailabilityServices.setJobGraphStore(jobGraphStore); - testingHighAvailabilityServices.setRunningJobsRegistry(runningJobsRegistry); + testingHighAvailabilityServices.setJobResultStore(jobResultStore); testingHighAvailabilityServices.setCloseFuture(closeFuture); testingHighAvailabilityServices.setCloseAndCleanupAllDataFuture( @@ -167,9 +167,8 @@ public TestingHighAvailabilityServicesBuilder setJobGraphStore(JobGraphStore job return this; } - public TestingHighAvailabilityServicesBuilder setRunningJobsRegistry( - RunningJobsRegistry runningJobsRegistry) { - this.runningJobsRegistry = runningJobsRegistry; + public TestingHighAvailabilityServicesBuilder setJobResultStore(JobResultStore jobResultStore) { + this.jobResultStore = jobResultStore; return this; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java index 6cbb92c8cc3fc..940c8d5331c0a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingManualHighAvailabilityServices.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; -import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.jobmanager.StandaloneJobGraphStore; import org.apache.flink.runtime.leaderelection.LeaderElectionService; @@ -117,8 +117,8 @@ public JobGraphStore getJobGraphStore() throws Exception { } @Override - public RunningJobsRegistry getRunningJobsRegistry() throws Exception { - return new StandaloneRunningJobsRegistry(); + public JobResultStore getJobResultStore() { + return new EmbeddedJobResultStore(); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStoreTest.java new file mode 100644 index 0000000000000..45644dc4381a5 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedJobResultStoreTest.java @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.highavailability.nonha.embedded; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.testutils.TestingJobResultStore; +import org.apache.flink.util.TestLoggerExtension; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.NoSuchElementException; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; +import static org.assertj.core.api.AssertionsForClassTypes.entry; + +/** Tests for the {@link EmbeddedJobResultStore}. */ +@ExtendWith(TestLoggerExtension.class) +public class EmbeddedJobResultStoreTest { + + private static final JobResultEntry DUMMY_JOB_RESULT_ENTRY = + new JobResultEntry(TestingJobResultStore.DUMMY_JOB_RESULT); + + private EmbeddedJobResultStore embeddedJobResultStore; + + @BeforeEach + public void setupTest() { + embeddedJobResultStore = new EmbeddedJobResultStore(); + } + + @Test + public void testStoreDirtyJobResult() throws Exception { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + assertThat(embeddedJobResultStore.hasJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())) + .isTrue(); + + assertThat(embeddedJobResultStore.dirtyJobResults) + .containsExactly(entry(DUMMY_JOB_RESULT_ENTRY.getJobId(), DUMMY_JOB_RESULT_ENTRY)); + assertThat(embeddedJobResultStore.cleanJobResults).isEmpty(); + } + + @Test + public void testStoreDirtyJobResultTwice() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + + assertThatThrownBy(() -> embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY)) + .isInstanceOf(IllegalStateException.class); + } + + @Test + public void testStoreDirtyJobResultForCleanJobEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + embeddedJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()); + + assertThatThrownBy(() -> embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY)) + .isInstanceOf(IllegalStateException.class); + } + + @Test + public void testCleanDirtyJobResult() throws Exception { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + embeddedJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()); + + assertThat(embeddedJobResultStore.dirtyJobResults).isEmpty(); + assertThat(embeddedJobResultStore.cleanJobResults) + .containsExactly(entry(DUMMY_JOB_RESULT_ENTRY.getJobId(), DUMMY_JOB_RESULT_ENTRY)); + } + + @Test + public void testCleanDirtyJobResultTwice() { + final JobID jobId = DUMMY_JOB_RESULT_ENTRY.getJobId(); + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + embeddedJobResultStore.markResultAsClean(jobId); + + assertThat(embeddedJobResultStore.cleanJobResults) + .containsExactly(entry(jobId, DUMMY_JOB_RESULT_ENTRY)); + embeddedJobResultStore.markResultAsClean(jobId); + assertThat(embeddedJobResultStore.cleanJobResults) + .as("Marking the same job %s as clean should be idempotent.", jobId) + .containsExactly(entry(jobId, DUMMY_JOB_RESULT_ENTRY)); + } + + @Test + public void testCleanNonExistentJobResult() throws Exception { + assertThatThrownBy(() -> embeddedJobResultStore.markResultAsClean(new JobID())) + .isInstanceOf(NoSuchElementException.class); + } + + @Test + public void testHasJobResultEntryWithNoEntry() { + assertThat(embeddedJobResultStore.hasJobResultEntry(new JobID())).isFalse(); + } + + @Test + public void testHasJobResultEntryWithDirtyEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + + assertThat(embeddedJobResultStore.hasJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())) + .isTrue(); + } + + @Test + public void testHasJobResultEntryWithCleanEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + embeddedJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()); + + assertThat(embeddedJobResultStore.hasJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())) + .isTrue(); + } + + @Test + public void testHasDirtyJobResultEntryWithNoDirtyEntry() { + assertThat(embeddedJobResultStore.hasDirtyJobResultEntry(new JobID())).isFalse(); + } + + @Test + public void testHasDirtyJobResultEntryWithDirtyEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + + assertThat(embeddedJobResultStore.hasDirtyJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())) + .isTrue(); + } + + @Test + public void testHasDirtyJobResultEntryWithCleanEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + embeddedJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()); + + assertThat(embeddedJobResultStore.hasDirtyJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())) + .isFalse(); + } + + @Test + public void testHasCleanJobResultEntryWithNoEntry() { + assertThat(embeddedJobResultStore.hasCleanJobResultEntry(new JobID())).isFalse(); + } + + @Test + public void testHasCleanJobResultEntryWithDirtyEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + assertThat(embeddedJobResultStore.hasCleanJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())) + .isFalse(); + } + + @Test + public void testHasCleanJobResultEntryWithCleanEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + embeddedJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()); + + assertThat(embeddedJobResultStore.hasCleanJobResultEntry(DUMMY_JOB_RESULT_ENTRY.getJobId())) + .isTrue(); + } + + @Test + public void testGetDirtyResultsWithNoEntry() { + assertThat(embeddedJobResultStore.getDirtyResults()).isEmpty(); + } + + @Test + public void testGetDirtyResultsWithDirtyEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + + assertThat(embeddedJobResultStore.getDirtyResults()) + .containsExactlyInAnyOrder(DUMMY_JOB_RESULT_ENTRY.getJobResult()); + } + + @Test + public void testGetDirtyResultsWithDirtyAndCleanEntry() { + embeddedJobResultStore.createDirtyResult(DUMMY_JOB_RESULT_ENTRY); + embeddedJobResultStore.markResultAsClean(DUMMY_JOB_RESULT_ENTRY.getJobId()); + + final JobResultEntry otherDirtyJobResultEntry = + new JobResultEntry(TestingJobResultStore.createSuccessfulJobResult(new JobID())); + embeddedJobResultStore.createDirtyResult(otherDirtyJobResultEntry); + + assertThat(embeddedJobResultStore.dirtyJobResults) + .containsExactly( + entry(otherDirtyJobResultEntry.getJobId(), otherDirtyJobResultEntry)); + assertThat(embeddedJobResultStore.cleanJobResults) + .containsExactly(entry(DUMMY_JOB_RESULT_ENTRY.getJobId(), DUMMY_JOB_RESULT_ENTRY)); + + assertThat(embeddedJobResultStore.getDirtyResults()) + .containsExactlyInAnyOrder(otherDirtyJobResultEntry.getJobResult()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServicesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServicesTest.java index 72b41a0ab633d..04556da7b2f62 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServicesTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperHaServicesTest.java @@ -23,7 +23,6 @@ import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.blob.BlobStoreService; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; import org.apache.flink.runtime.leaderelection.LeaderElectionService; import org.apache.flink.runtime.leaderelection.TestingContender; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -174,7 +173,7 @@ public void testCleanupJobData() throws Exception { haServices -> { final List childrenBefore = client.getChildren().forPath(path); - haServices.cleanupJobData(jobID); + haServices.globalCleanup(jobID); final List childrenAfter = client.getChildren().forPath(path); @@ -235,9 +234,6 @@ private void runCleanupTestWithJob( final LeaderElectionService jobManagerLeaderElectionService = zooKeeperHaServices.getJobManagerLeaderElectionService(jobId); - final RunningJobsRegistry runningJobsRegistry = - zooKeeperHaServices.getRunningJobsRegistry(); - final LeaderRetrievalUtils.LeaderConnectionInfoListener resourceManagerLeaderListener = new LeaderRetrievalUtils.LeaderConnectionInfoListener(); resourceManagerLeaderElectionService.start( @@ -253,8 +249,6 @@ private void runCleanupTestWithJob( "unused-jobmanager-address", jobManagerLeaderElectionService)); jobManagerLeaderRetriever.start(jobManagerLeaderListener); - runningJobsRegistry.setJobRunning(jobId); - // Make sure that the respective zNodes have been properly created resourceManagerLeaderListener.getLeaderConnectionInfoFuture().join(); jobManagerLeaderListener.getLeaderConnectionInfoFuture().join(); @@ -263,7 +257,6 @@ private void runCleanupTestWithJob( resourceManagerLeaderElectionService.stop(); jobManagerLeaderRetriever.stop(); jobManagerLeaderElectionService.stop(); - runningJobsRegistry.clearJob(jobId); zooKeeperHaServicesConsumer.accept(zooKeeperHaServices); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRegistryTest.java deleted file mode 100644 index 7849274500a52..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperRegistryTest.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.highavailability.zookeeper; - -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.HighAvailabilityOptions; -import org.apache.flink.runtime.blob.VoidBlobStore; -import org.apache.flink.runtime.highavailability.HighAvailabilityServices; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry.JobSchedulingStatus; -import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler; -import org.apache.flink.runtime.util.ZooKeeperUtils; -import org.apache.flink.util.TestLogger; -import org.apache.flink.util.concurrent.Executors; - -import org.apache.curator.test.TestingServer; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import static org.junit.Assert.assertEquals; - -public class ZooKeeperRegistryTest extends TestLogger { - - private TestingServer testingServer; - - @Before - public void before() throws Exception { - testingServer = new TestingServer(); - } - - @After - public void after() throws Exception { - testingServer.stop(); - testingServer = null; - } - - /** - * Tests that the function of ZookeeperRegistry, setJobRunning(), setJobFinished(), - * isJobRunning(). - */ - @Test - public void testZooKeeperRegistry() throws Exception { - Configuration configuration = new Configuration(); - configuration.setString( - HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, testingServer.getConnectString()); - configuration.setString(HighAvailabilityOptions.HA_MODE, "zookeeper"); - - final HighAvailabilityServices zkHaService = - new ZooKeeperHaServices( - ZooKeeperUtils.startCuratorFramework( - configuration, NoOpFatalErrorHandler.INSTANCE), - Executors.directExecutor(), - configuration, - new VoidBlobStore()); - - final RunningJobsRegistry zkRegistry = zkHaService.getRunningJobsRegistry(); - - try { - JobID jobID = JobID.generate(); - assertEquals(JobSchedulingStatus.PENDING, zkRegistry.getJobSchedulingStatus(jobID)); - - // set when znode does not exist for job - zkRegistry.setJobRunning(jobID); - assertEquals(JobSchedulingStatus.RUNNING, zkRegistry.getJobSchedulingStatus(jobID)); - - // set when znode does exist for job - zkRegistry.setJobFinished(jobID); - assertEquals(JobSchedulingStatus.DONE, zkRegistry.getJobSchedulingStatus(jobID)); - - zkRegistry.clearJob(jobID); - assertEquals(JobSchedulingStatus.PENDING, zkRegistry.getJobSchedulingStatus(jobID)); - - // clear when znode does not exist for job - zkRegistry.clearJob(jobID); - } finally { - zkHaService.closeAndCleanupAllData(); - } - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java index c53717edb514a..f3074f9682b94 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/CreditBasedPartitionRequestClientHandlerTest.java @@ -405,8 +405,8 @@ public void testNotifyCreditAvailable() throws Exception { inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannels[0].requestSubpartition(0); - inputChannels[1].requestSubpartition(0); + inputChannels[0].requestSubpartition(); + inputChannels[1].requestSubpartition(); // The two input channels should send partition requests assertTrue(channel.isWritable()); @@ -532,7 +532,7 @@ public void testNotifyCreditAvailableAfterReleased() throws Exception { inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); // This should send the partition request Object readFromOutbound = channel.readOutbound(); @@ -699,8 +699,8 @@ public void testAnnounceBufferSize() throws Exception { inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannels[0].requestSubpartition(0); - inputChannels[1].requestSubpartition(0); + inputChannels[0].requestSubpartition(); + inputChannels[1].requestSubpartition(); channel.readOutbound(); channel.readOutbound(); @@ -827,6 +827,7 @@ private static class TestRemoteInputChannelForError extends RemoteInputChannel { inputGate, 0, new ResultPartitionID(), + 0, InputChannelBuilder.STUB_CONNECTION_ID, new TestingConnectionManager(), 0, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java index 4931a2f934fe2..801e478b3404c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientDecoderDelegateTest.java @@ -85,7 +85,7 @@ public void setup() throws IOException, InterruptedException { inputGate, new TestingPartitionRequestClient(), NUMBER_OF_BUFFER_RESPONSES); inputGate.setInputChannels(inputChannel); inputGate.setup(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); handler.addInputChannel(inputChannel); inputChannelId = inputChannel.getInputChannelId(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java index cba3be2575feb..ee42d8dd748a2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageClientSideSerializationTest.java @@ -84,7 +84,7 @@ public void setup() throws IOException, InterruptedException { inputGate = createSingleInputGate(1, networkBufferPool); RemoteInputChannel inputChannel = createRemoteInputChannel(inputGate, new TestingPartitionRequestClient()); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); inputGate.setInputChannels(inputChannel); inputGate.setup(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java index 3d8e8d5658bce..276a2e2828419 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClientTest.java @@ -78,7 +78,7 @@ public void testRetriggerPartitionRequest() throws Exception { inputGate.setupChannels(); // first subpartition request - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); assertTrue(channel.isWritable()); Object readFromOutbound = channel.readOutbound(); @@ -89,7 +89,9 @@ public void testRetriggerPartitionRequest() throws Exception { assertEquals(numExclusiveBuffers, ((PartitionRequest) readFromOutbound).credit); // retrigger subpartition request, e.g. due to failures - inputGate.retriggerPartitionRequest(inputChannel.getPartitionId().getPartitionId()); + inputGate.retriggerPartitionRequest( + inputChannel.getPartitionId().getPartitionId(), + inputChannel.getConsumedSubpartitionIndex()); runAllScheduledPendingTasks(channel, deadline); readFromOutbound = channel.readOutbound(); @@ -100,7 +102,9 @@ public void testRetriggerPartitionRequest() throws Exception { assertEquals(numExclusiveBuffers, ((PartitionRequest) readFromOutbound).credit); // retrigger subpartition request once again, e.g. due to failures - inputGate.retriggerPartitionRequest(inputChannel.getPartitionId().getPartitionId()); + inputGate.retriggerPartitionRequest( + inputChannel.getPartitionId().getPartitionId(), + inputChannel.getConsumedSubpartitionIndex()); runAllScheduledPendingTasks(channel, deadline); readFromOutbound = channel.readOutbound(); @@ -137,7 +141,7 @@ public void testDoublePartitionRequest() throws Exception { final BufferPool bufferPool = networkBufferPool.createBufferPool(6, 6); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); // The input channel should only send one partition request assertTrue(channel.isWritable()); @@ -173,7 +177,7 @@ public void testResumeConsumption() throws Exception { final BufferPool bufferPool = networkBufferPool.createBufferPool(6, 6); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); inputChannel.resumeConsumption(); channel.runPendingTasks(); @@ -211,7 +215,7 @@ public void testAcknowledgeAllRecordsProcessed() throws Exception { BufferPool bufferPool = networkBufferPool.createBufferPool(6, 6); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); inputChannel.acknowledgeAllRecordsProcessed(); channel.runPendingTasks(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java index 3fdab4419fe28..8af69be7e64c1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/InputGateFairnessTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.io.network.partition; +import org.apache.flink.runtime.deployment.SubpartitionIndexRange; import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; @@ -304,7 +305,10 @@ public void testFairConsumptionRemoteChannels() throws Exception { private SingleInputGate createFairnessVerifyingInputGate(int numberOfChannels) { return new FairnessVerifyingInputGate( - "Test Task Name", new IntermediateDataSetID(), 0, numberOfChannels); + "Test Task Name", + new IntermediateDataSetID(), + new SubpartitionIndexRange(0, 0), + numberOfChannels); } private void fillRandom( @@ -362,7 +366,7 @@ private static class FairnessVerifyingInputGate extends SingleInputGate { public FairnessVerifyingInputGate( String owningTaskName, IntermediateDataSetID consumedResultId, - int consumedSubpartitionIndex, + SubpartitionIndexRange subpartitionIndexRange, int numberOfInputChannels) { super( @@ -370,7 +374,7 @@ public FairnessVerifyingInputGate( 0, consumedResultId, ResultPartitionType.PIPELINED, - consumedSubpartitionIndex, + subpartitionIndexRange, numberOfInputChannels, SingleInputGateBuilder.NO_OP_PRODUCER_CHECKER, STUB_BUFFER_POOL_FACTORY, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java index 46506307c92c0..090a4323ad0d8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SortMergeResultPartitionReadSchedulerTest.java @@ -92,7 +92,9 @@ public void before() throws Exception { dataBytes); bufferPool = new BatchShuffleReadBufferPool(totalBytes, bufferSize); executor = Executors.newFixedThreadPool(numThreads); - readScheduler = new SortMergeResultPartitionReadScheduler(bufferPool, executor, this); + readScheduler = + new SortMergeResultPartitionReadScheduler( + numSubpartitions, bufferPool, executor, this); } @After @@ -209,7 +211,7 @@ public void testRequestBufferTimeoutAndFailed() throws Exception { List buffers = bufferPool.requestBuffers(); SortMergeResultPartitionReadScheduler readScheduler = new SortMergeResultPartitionReadScheduler( - bufferPool, executor, this, bufferRequestTimeout); + numSubpartitions, bufferPool, executor, this, bufferRequestTimeout); SortMergeSubpartitionReader subpartitionReader = readScheduler.createSubpartitionReader( @@ -237,7 +239,7 @@ public void testRequestTimeoutIsRefreshedAndSuccess() throws Exception { new FakeBatchShuffleReadBufferPool(bufferSize * 3, bufferSize); SortMergeResultPartitionReadScheduler readScheduler = new SortMergeResultPartitionReadScheduler( - bufferPool, executor, this, bufferRequestTimeout); + numSubpartitions, bufferPool, executor, this, bufferRequestTimeout); FileChannel dataFileChannel = openFileChannel(partitionedFile.getDataFilePath()); FileChannel indexFileChannel = openFileChannel(partitionedFile.getIndexFilePath()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java index 3da8e21d18762..7d4b434995c0d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputBuffersMetricsTest.java @@ -215,7 +215,7 @@ public void testFloatingBuffersUsage() throws Exception { extraNetworkBuffersPerGate * inputGates.length + buffersPerChannel * totalNumberOfRemoteChannels; - remoteInputChannel1.requestSubpartition(0); + remoteInputChannel1.requestSubpartition(); int backlog = 3; int totalRequestedBuffers = buffersPerChannel + backlog; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java index cb6e975abf578..bb5d24a736b85 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java @@ -42,6 +42,7 @@ public class InputChannelBuilder { private int channelIndex = 0; private ResultPartitionID partitionId = new ResultPartitionID(); + private int consumedSubpartitionIndex = 0; private ConnectionID connectionID = STUB_CONNECTION_ID; private ResultPartitionManager partitionManager = new TestingResultPartitionManager(new NoOpResultSubpartitionView()); @@ -68,6 +69,11 @@ public InputChannelBuilder setPartitionId(ResultPartitionID partitionId) { return this; } + public InputChannelBuilder setConsumedSubpartitionIndex(int consumedSubpartitionIndex) { + this.consumedSubpartitionIndex = consumedSubpartitionIndex; + return this; + } + public InputChannelBuilder setPartitionManager(ResultPartitionManager partitionManager) { this.partitionManager = partitionManager; return this; @@ -123,6 +129,7 @@ UnknownInputChannel buildUnknownChannel(SingleInputGate inputGate) { inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, partitionManager, taskEventPublisher, connectionManager, @@ -139,6 +146,7 @@ public LocalInputChannel buildLocalChannel(SingleInputGate inputGate) { inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, partitionManager, taskEventPublisher, initialBackoff, @@ -153,6 +161,7 @@ public RemoteInputChannel buildRemoteChannel(SingleInputGate inputGate) { inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, connectionID, connectionManager, initialBackoff, @@ -169,6 +178,7 @@ public LocalRecoveredInputChannel buildLocalRecoveredChannel(SingleInputGate inp inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, partitionManager, taskEventPublisher, initialBackoff, @@ -185,6 +195,7 @@ public RemoteRecoveredInputChannel buildRemoteRecoveredChannel(SingleInputGate i inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, connectionID, connectionManager, initialBackoff, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java index d6de0ea72f035..d8757331742ac 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelTest.java @@ -126,6 +126,7 @@ private MockInputChannel( inputGate, channelIndex, partitionId, + 0, initialBackoff, maxBackoff, new SimpleCounter(), @@ -139,7 +140,7 @@ public void resumeConsumption() {} public void acknowledgeAllRecordsProcessed() throws IOException {} @Override - void requestSubpartition(int subpartitionIndex) throws IOException, InterruptedException {} + void requestSubpartition() throws IOException, InterruptedException {} @Override Optional getNextBuffer() throws IOException, InterruptedException { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java index 30f355ab8ab57..1df9fc556a27b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java @@ -116,7 +116,7 @@ public void testNoDataPersistedAfterReceivingAlignedBarrier() throws Exception { barrierHolder, data))) .setStateWriter(stateWriter) .buildLocalChannel(new SingleInputGateBuilder().build()); - channel.requestSubpartition(0); + channel.requestSubpartition(); // pull AC barrier channel.getNextBuffer(); @@ -261,21 +261,21 @@ public void testPartitionRequestExponentialBackoff() throws Exception { .schedule(any(TimerTask.class), anyLong()); // Initial request - ch.requestSubpartition(0); + ch.requestSubpartition(); verify(partitionManager) .createSubpartitionView( eq(ch.partitionId), eq(0), any(BufferAvailabilityListener.class)); // Request subpartition and verify that the actual requests are delayed. for (long expected : expectedDelays) { - ch.retriggerSubpartitionRequest(timer, 0); + ch.retriggerSubpartitionRequest(timer); verify(timer).schedule(any(TimerTask.class), eq(expected)); } // Exception after backoff is greater than the maximum backoff. try { - ch.retriggerSubpartitionRequest(timer, 0); + ch.retriggerSubpartitionRequest(timer); ch.getNextBuffer(); fail("Did not throw expected exception."); } catch (Exception expected) { @@ -301,14 +301,14 @@ public void testProducerFailedException() throws Exception { LocalInputChannel ch = createLocalInputChannel(inputGate, partitionManager); - ch.requestSubpartition(0); + ch.requestSubpartition(); // Should throw an instance of CancelTaskException. ch.getNextBuffer(); } /** - * Tests that {@link LocalInputChannel#requestSubpartition(int)} throws {@link + * Tests that {@link LocalInputChannel#requestSubpartition()} throws {@link * PartitionNotFoundException} if the result partition was not registered in {@link * ResultPartitionManager} and no backoff. */ @@ -319,7 +319,7 @@ public void testPartitionNotFoundExceptionWhileRequestingPartition() throws Exce createLocalInputChannel(inputGate, new ResultPartitionManager()); try { - localChannel.requestSubpartition(0); + localChannel.requestSubpartition(); fail("Should throw a PartitionNotFoundException."); } catch (PartitionNotFoundException notFound) { @@ -329,7 +329,7 @@ public void testPartitionNotFoundExceptionWhileRequestingPartition() throws Exce /** * Tests that {@link SingleInputGate#retriggerPartitionRequest(IntermediateResultPartitionID)} - * is triggered after {@link LocalInputChannel#requestSubpartition(int)} throws {@link + * is triggered after {@link LocalInputChannel#requestSubpartition()} throws {@link * PartitionNotFoundException} within backoff. */ @Test @@ -339,15 +339,15 @@ public void testRetriggerPartitionRequestWhilePartitionNotFound() throws Excepti createLocalInputChannel(inputGate, new ResultPartitionManager(), 1, 1); inputGate.setInputChannels(localChannel); - localChannel.requestSubpartition(0); + localChannel.requestSubpartition(); // The timer should be initialized at the first time of retriggering partition request. assertNotNull(inputGate.getRetriggerLocalRequestTimer()); } /** - * Tests that {@link LocalInputChannel#retriggerSubpartitionRequest(Timer, int)} would throw - * {@link PartitionNotFoundException} which is set onto the input channel then. + * Tests that {@link LocalInputChannel#retriggerSubpartitionRequest(Timer)} would throw {@link + * PartitionNotFoundException} which is set onto the input channel then. */ @Test public void testChannelErrorWhileRetriggeringRequest() { @@ -376,7 +376,7 @@ public void schedule(TimerTask task, long delay) { }; try { - localChannel.retriggerSubpartitionRequest(timer, 0); + localChannel.retriggerSubpartitionRequest(timer); } finally { timer.cancel(); } @@ -437,7 +437,7 @@ public void testConcurrentReleaseAndRetriggerPartitionRequest() throws Exception new Thread( () -> { try { - channel.requestSubpartition(0); + channel.requestSubpartition(); } catch (IOException ignored) { } }); @@ -462,7 +462,7 @@ public void testGetNextAfterPartitionReleased() throws Exception { LocalInputChannel channel = createLocalInputChannel(new SingleInputGateBuilder().build(), partitionManager); - channel.requestSubpartition(0); + channel.requestSubpartition(); assertFalse(channel.getNextBuffer().isPresent()); // release the subpartition view @@ -489,7 +489,7 @@ public void testGetBufferFromLocalChannelWhenCompressionEnabled() throws Excepti createLocalInputChannel(new SingleInputGateBuilder().build(), partitionManager); // request partition and get next buffer - channel.requestSubpartition(0); + channel.requestSubpartition(); Optional bufferAndAvailability = channel.getNextBuffer(); assertTrue(bufferAndAvailability.isPresent()); @@ -515,7 +515,7 @@ public void testAnnounceBufferSize() throws Exception { InputChannelTestUtils.createResultSubpartitionView(true)); SingleInputGate inputGate = createSingleInputGate(1); LocalInputChannel localChannel = createLocalInputChannel(inputGate, partitionManager); - localChannel.requestSubpartition(0); + localChannel.requestSubpartition(); localChannel.announceBufferSize(10); @@ -539,7 +539,7 @@ public void testEnqueueAvailableChannelWhenResuming() throws IOException, Interr new TestingResultPartitionManager(subpartitionView); LocalInputChannel channel = createLocalInputChannel(new SingleInputGateBuilder().build(), partitionManager); - channel.requestSubpartition(0); + channel.requestSubpartition(); // Block the subpartition subpartition.add( @@ -581,7 +581,7 @@ public void testCheckpointingInflightData() throws Exception { createLocalInputChannel( inputGate, partitionManager, 0, 0, b -> b.setStateWriter(stateWriter)); inputGate.setInputChannels(channel); - channel.requestSubpartition(0); + channel.requestSubpartition(); final CheckpointStorageLocationReference location = getDefault(); CheckpointOptions options = @@ -631,8 +631,8 @@ public void testAnnounceNewBufferSize() throws IOException, InterruptedException new SingleInputGateBuilder().build(), new TestingResultPartitionManager(subpartition1.createReadView(() -> {}))); - channel0.requestSubpartition(0); - channel1.requestSubpartition(0); + channel0.requestSubpartition(); + channel1.requestSubpartition(); // and: Preferable buffer size is default value. assertEquals(Integer.MAX_VALUE, subpartition0.add(createFilledFinishedBufferConsumer(16))); @@ -666,7 +666,7 @@ public void testReceivingBuffersInUseBeforeSubpartitionViewInitialization() thro assertEquals(0, localChannel.getBuffersInUseCount()); // when: The subpartition view is initialized. - localChannel.requestSubpartition(0); + localChannel.requestSubpartition(); // then: Buffers in use should show correct value. assertEquals(3, localChannel.getBuffersInUseCount()); @@ -737,7 +737,6 @@ public TestLocalInputChannelConsumer( this.inputGate = new SingleInputGateBuilder() - .setConsumedSubpartitionIndex(subpartitionIndex) .setNumberOfChannels(numberOfInputChannels) .setBufferPoolFactory(bufferPool) .build(); @@ -748,6 +747,7 @@ public TestLocalInputChannelConsumer( inputChannels[i] = InputChannelBuilder.newBuilder() .setChannelIndex(i) + .setConsumedSubpartitionIndex(subpartitionIndex) .setPartitionManager(partitionManager) .setPartitionId(consumedPartitionIds[i]) .setTaskEventPublisher(taskEventDispatcher) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannelTest.java index 99a562b50ef08..eb3fb01180a15 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannelTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/RecoveredInputChannelTest.java @@ -41,7 +41,7 @@ public void testConversionOnlyPossibleAfterConsumed() throws IOException { @Test(expected = UnsupportedOperationException.class) public void testRequestPartitionsImpossible() { - buildChannel().requestSubpartition(0); + buildChannel().requestSubpartition(); } @Test(expected = CheckpointException.class) @@ -60,6 +60,7 @@ private RecoveredInputChannel buildChannel() { new ResultPartitionID(), 0, 0, + 0, new SimpleCounter(), new SimpleCounter(), 10) { 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 942d91caa6642..71d1e5f2a1ce6 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 @@ -139,7 +139,7 @@ public void testGateNotifiedOnBarrierConversion() throws IOException, Interrupte new TestVerifyConnectionManager( new TestVerifyPartitionRequestClient())) .buildRemoteChannel(inputGate); - channel.requestSubpartition(0); + channel.requestSubpartition(); channel.onBuffer( toBuffer( @@ -342,9 +342,9 @@ private void testConcurrentReleaseAndSomething( public void testRetriggerWithoutPartitionRequest() throws Exception { SingleInputGate inputGate = createSingleInputGate(1); - RemoteInputChannel ch = createRemoteInputChannel(inputGate, 500, 3000); + RemoteInputChannel ch = createRemoteInputChannel(inputGate, 0, 500, 3000); - ch.retriggerSubpartitionRequest(0); + ch.retriggerSubpartitionRequest(); } @Test @@ -361,19 +361,19 @@ public void testPartitionRequestExponentialBackoff() throws Exception { createRemoteInputChannel(inputGate, connectionManager, partitionId, 500, 3000); // Initial request - ch.requestSubpartition(0); + ch.requestSubpartition(); client.verifyResult(partitionId, 0, 0); // Request subpartition and verify that the actual requests are delayed. for (int expected : expectedDelays) { - ch.retriggerSubpartitionRequest(0); + ch.retriggerSubpartitionRequest(); client.verifyResult(partitionId, 0, expected); } // Exception after backoff is greater than the maximum backoff. try { - ch.retriggerSubpartitionRequest(0); + ch.retriggerSubpartitionRequest(); ch.getNextBuffer(); fail("Did not throw expected exception."); } catch (Exception expected) { @@ -391,16 +391,16 @@ public void testPartitionRequestSingleBackoff() throws Exception { createRemoteInputChannel(inputGate, connectionManager, partitionId, 500, 500); // No delay for first request - ch.requestSubpartition(0); + ch.requestSubpartition(); client.verifyResult(partitionId, 0, 0); // Initial delay for second request - ch.retriggerSubpartitionRequest(0); + ch.retriggerSubpartitionRequest(); client.verifyResult(partitionId, 0, 500); // Exception after backoff is greater than the maximum backoff. try { - ch.retriggerSubpartitionRequest(0); + ch.retriggerSubpartitionRequest(); ch.getNextBuffer(); fail("Did not throw expected exception."); } catch (Exception expected) { @@ -418,12 +418,12 @@ public void testPartitionRequestNoBackoff() throws Exception { createRemoteInputChannel(inputGate, connectionManager, partitionId, 0, 0); // No delay for first request - ch.requestSubpartition(0); + ch.requestSubpartition(); client.verifyResult(partitionId, 0, 0); // Exception, because backoff is disabled. try { - ch.retriggerSubpartitionRequest(0); + ch.retriggerSubpartitionRequest(); ch.getNextBuffer(); fail("Did not throw expected exception."); } catch (Exception expected) { @@ -460,7 +460,7 @@ public void testProducerFailedException() throws Exception { ch.onError(new ProducerFailedException(new RuntimeException("Expected test exception."))); - ch.requestSubpartition(0); + ch.requestSubpartition(); // Should throw an instance of CancelTaskException. ch.getNextBuffer(); @@ -504,7 +504,7 @@ public void testAvailableBuffersLessThanRequiredBuffers() throws Exception { spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers)); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); // Prepare the exclusive and floating buffers to verify recycle logic later final Buffer exclusiveBuffer = inputChannel.requestBuffer(); @@ -692,7 +692,7 @@ public void testAvailableBuffersEqualToRequiredBuffers() throws Exception { spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers)); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); // Prepare the exclusive and floating buffers to verify recycle logic later final Buffer exclusiveBuffer = inputChannel.requestBuffer(); @@ -788,7 +788,7 @@ public void testAvailableBuffersMoreThanRequiredBuffers() throws Exception { spy(networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers)); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); // Prepare the exclusive and floating buffers to verify recycle logic later final Buffer exclusiveBuffer = inputChannel.requestBuffer(); @@ -911,7 +911,7 @@ public void testFairDistributionFloatingBuffers() throws Exception { inputGate.setupChannels(); inputGate.requestPartitions(); for (RemoteInputChannel inputChannel : inputChannels) { - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); } // Exhaust all the floating buffers @@ -972,7 +972,7 @@ public void testFailureInNotifyBufferAvailable() throws Exception { final SingleInputGate inputGate = createSingleInputGate(1); final RemoteInputChannel successfulRemoteIC = createRemoteInputChannel(inputGate); - successfulRemoteIC.requestSubpartition(0); + successfulRemoteIC.requestSubpartition(); // late creation -> no exclusive buffers, also no requested subpartition in // successfulRemoteIC @@ -1041,7 +1041,7 @@ public void testConcurrentOnSenderBacklogAndRelease() throws Exception { networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); final Callable requestBufferTask = new Callable() { @@ -1113,7 +1113,7 @@ public void testConcurrentOnSenderBacklogAndRecycle() throws Exception { networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); final Callable requestBufferTask = new Callable() { @@ -1177,7 +1177,7 @@ public void testConcurrentRecycleAndRelease() throws Exception { networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); final Callable releaseTask = new Callable() { @@ -1244,7 +1244,7 @@ public void testConcurrentRecycleAndRelease2() throws Exception { networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); final Callable bufferPoolInteractionsTask = () -> { @@ -1326,7 +1326,7 @@ public void testConcurrentGetNextBufferAndRelease() throws Exception { networkBufferPool.createBufferPool(numFloatingBuffers, numFloatingBuffers); inputGate.setBufferPool(bufferPool); inputGate.setupChannels(); - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); for (int i = 0; i < numTotalBuffers; i++) { Buffer buffer = inputChannel.requestBuffer(); @@ -1367,8 +1367,8 @@ public void testConcurrentGetNextBufferAndRelease() throws Exception { } /** - * Tests that {@link RemoteInputChannel#retriggerSubpartitionRequest(int)} would throw the - * {@link PartitionNotFoundException} if backoff is 0. + * Tests that {@link RemoteInputChannel#retriggerSubpartitionRequest()} would throw the {@link + * PartitionNotFoundException} if backoff is 0. */ @Test public void testPartitionNotFoundExceptionWhileRetriggeringRequest() throws Exception { @@ -1378,9 +1378,9 @@ public void testPartitionNotFoundExceptionWhileRetriggeringRequest() throws Exce // Request partition to initialize client to avoid illegal state after retriggering // partition - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); // The default backoff is 0 then it would set PartitionNotFoundException on this channel - inputChannel.retriggerSubpartitionRequest(0); + inputChannel.retriggerSubpartitionRequest(); try { inputChannel.checkError(); @@ -1393,7 +1393,7 @@ public void testPartitionNotFoundExceptionWhileRetriggeringRequest() throws Exce /** * Tests that any exceptions thrown by {@link * ConnectionManager#createPartitionRequestClient(ConnectionID)} would be wrapped into {@link - * PartitionConnectionException} during {@link RemoteInputChannel#requestSubpartition(int)}. + * PartitionConnectionException} during {@link RemoteInputChannel#requestSubpartition()}. */ @Test public void testPartitionConnectionExceptionWhileRequestingPartition() throws Exception { @@ -1401,7 +1401,7 @@ public void testPartitionConnectionExceptionWhileRequestingPartition() throws Ex InputChannelTestUtils.createRemoteInputChannel( createSingleInputGate(1), 0, new TestingExceptionConnectionManager()); try { - inputChannel.requestSubpartition(0); + inputChannel.requestSubpartition(); fail("Expected PartitionConnectionException."); } catch (PartitionConnectionException ex) { assertThat(inputChannel.getPartitionId(), is(ex.getPartitionId())); @@ -1431,11 +1431,11 @@ public void testOnUpstreamBlockedAndResumed() throws Exception { BufferPool bufferPool = new TestBufferPool(); SingleInputGate inputGate = createSingleInputGate(bufferPool); - RemoteInputChannel remoteChannel1 = createRemoteInputChannel(inputGate, 2); - RemoteInputChannel remoteChannel2 = createRemoteInputChannel(inputGate, 0); + RemoteInputChannel remoteChannel1 = createRemoteInputChannel(inputGate, 0, 2); + RemoteInputChannel remoteChannel2 = createRemoteInputChannel(inputGate, 1, 0); inputGate.setup(); - remoteChannel1.requestSubpartition(0); - remoteChannel2.requestSubpartition(1); + remoteChannel1.requestSubpartition(); + remoteChannel2.requestSubpartition(); remoteChannel1.onSenderBacklog(2); remoteChannel2.onSenderBacklog(2); @@ -1480,11 +1480,11 @@ public void testRequestBuffer() throws Exception { BufferPool bufferPool = new TestBufferPool(); SingleInputGate inputGate = createSingleInputGate(bufferPool); - RemoteInputChannel remoteChannel1 = createRemoteInputChannel(inputGate, 2); - RemoteInputChannel remoteChannel2 = createRemoteInputChannel(inputGate, 0); + RemoteInputChannel remoteChannel1 = createRemoteInputChannel(inputGate, 0, 2); + RemoteInputChannel remoteChannel2 = createRemoteInputChannel(inputGate, 1, 0); inputGate.setup(); - remoteChannel1.requestSubpartition(0); - remoteChannel2.requestSubpartition(1); + remoteChannel1.requestSubpartition(); + remoteChannel2.requestSubpartition(); remoteChannel1.onSenderBacklog(2); remoteChannel2.onSenderBacklog(2); @@ -1722,19 +1722,24 @@ private void assertGetNextBufferSequenceNumbers( // --------------------------------------------------------------------------------------------- private RemoteInputChannel createRemoteInputChannel(SingleInputGate inputGate) { - return createRemoteInputChannel(inputGate, 0, 0); + return createRemoteInputChannel(inputGate, 0, 0, 0); } private RemoteInputChannel createRemoteInputChannel( - SingleInputGate inputGate, int initialCredits) { + SingleInputGate inputGate, int consumedSubpartitionIndex, int initialCredits) { return InputChannelBuilder.newBuilder() + .setConsumedSubpartitionIndex(consumedSubpartitionIndex) .setNetworkBuffersPerChannel(initialCredits) .buildRemoteChannel(inputGate); } private RemoteInputChannel createRemoteInputChannel( - SingleInputGate inputGate, int initialBackoff, int maxBackoff) { + SingleInputGate inputGate, + int consumedSubpartitionIndex, + int initialBackoff, + int maxBackoff) { return InputChannelBuilder.newBuilder() + .setConsumedSubpartitionIndex(consumedSubpartitionIndex) .setInitialBackoff(initialBackoff) .setMaxBackoff(maxBackoff) .buildRemoteChannel(inputGate); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java index 3ee0a748aabdd..c7e1faa82c37f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java @@ -21,6 +21,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.MemorySegmentProvider; import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter; +import org.apache.flink.runtime.deployment.SubpartitionIndexRange; import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.buffer.BufferDecompressor; import org.apache.flink.runtime.io.network.buffer.BufferPool; @@ -55,7 +56,7 @@ public class SingleInputGateBuilder { private ResultPartitionType partitionType = ResultPartitionType.PIPELINED; - private int consumedSubpartitionIndex = 0; + private SubpartitionIndexRange subpartitionIndexRange = new SubpartitionIndexRange(0, 0); private int gateIndex = 0; @@ -91,8 +92,9 @@ public SingleInputGateBuilder setResultPartitionType(ResultPartitionType partiti return this; } - public SingleInputGateBuilder setConsumedSubpartitionIndex(int consumedSubpartitionIndex) { - this.consumedSubpartitionIndex = consumedSubpartitionIndex; + public SingleInputGateBuilder setSubpartitionIndexRange( + SubpartitionIndexRange subpartitionIndexRange) { + this.subpartitionIndexRange = subpartitionIndexRange; return this; } @@ -161,7 +163,7 @@ public SingleInputGate build() { gateIndex, intermediateDataSetID, partitionType, - consumedSubpartitionIndex, + subpartitionIndexRange, numberOfChannels, partitionProducerStateProvider, bufferPoolFactory, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index abf1d58abdd58..1dd28ece36ea1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -26,9 +26,12 @@ import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; +import org.apache.flink.runtime.deployment.SubpartitionIndexRange; +import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.PullingAsyncDataInput; +import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.TaskEventDispatcher; @@ -46,6 +49,8 @@ import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener; import org.apache.flink.runtime.io.network.partition.BufferWritingResultPartition; +import org.apache.flink.runtime.io.network.partition.ChannelStateHolder; +import org.apache.flink.runtime.io.network.partition.InputChannelTestUtils; import org.apache.flink.runtime.io.network.partition.NoOpResultSubpartitionView; import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; import org.apache.flink.runtime.io.network.partition.ResultPartition; @@ -54,6 +59,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; +import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.SubpartitionInfo; import org.apache.flink.runtime.io.network.util.TestTaskEvent; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; @@ -62,6 +68,7 @@ import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.shuffle.UnknownShuffleDescriptor; +import org.apache.flink.util.CompressedSerializedValue; import org.apache.flink.shaded.guava30.com.google.common.io.Closer; @@ -73,6 +80,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; @@ -83,7 +91,6 @@ import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createLocalInputChannel; import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createRemoteInputChannel; import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createResultSubpartitionView; -import static org.apache.flink.runtime.io.network.partition.InputChannelTestUtils.createSingleInputGate; import static org.apache.flink.runtime.io.network.partition.InputGateFairnessTest.setupInputGate; import static org.apache.flink.runtime.io.network.util.TestBufferFactory.createBuffer; import static org.apache.flink.runtime.state.CheckpointStorageLocationReference.getDefault; @@ -574,25 +581,6 @@ public void testRequestBackoffConfiguration() throws Exception { new IntermediateResultPartitionID() }; - ResourceID localLocation = ResourceID.generate(); - ShuffleDescriptor[] channelDescs = - new ShuffleDescriptor[] { - // Local - createRemoteWithIdAndLocation(partitionIds[0], localLocation), - // Remote - createRemoteWithIdAndLocation(partitionIds[1], ResourceID.generate()), - // Unknown - new UnknownShuffleDescriptor( - new ResultPartitionID(partitionIds[2], new ExecutionAttemptID())) - }; - - InputGateDeploymentDescriptor gateDesc = - new InputGateDeploymentDescriptor( - new IntermediateDataSetID(), - ResultPartitionType.PIPELINED, - 0, - channelDescs); - int initialBackoff = 137; int maxBackoff = 1001; @@ -602,22 +590,8 @@ public void testRequestBackoffConfiguration() throws Exception { .setPartitionRequestMaxBackoff(maxBackoff) .build(); - final TaskMetricGroup taskMetricGroup = - UnregisteredMetricGroups.createUnregisteredTaskMetricGroup(); SingleInputGate gate = - new SingleInputGateFactory( - localLocation, - netEnv.getConfiguration(), - netEnv.getConnectionManager(), - netEnv.getResultPartitionManager(), - new TaskEventDispatcher(), - netEnv.getNetworkBufferPool()) - .create( - netEnv.createShuffleIOOwnerContext( - "TestTask", taskMetricGroup.executionId(), taskMetricGroup), - 0, - gateDesc, - SingleInputGateBuilder.NO_OP_PRODUCER_CHECKER); + createSingleInputGate(partitionIds, ResultPartitionType.PIPELINED, netEnv); gate.setChannelStateWriter(ChannelStateWriter.NO_OP); gate.finishReadRecoveredState(); @@ -630,9 +604,9 @@ public void testRequestBackoffConfiguration() throws Exception { closer.register(netEnv::close); closer.register(gate::close); - assertEquals(gateDesc.getConsumedPartitionType(), gate.getConsumedPartitionType()); + assertEquals(ResultPartitionType.PIPELINED, gate.getConsumedPartitionType()); - Map channelMap = gate.getInputChannels(); + Map channelMap = gate.getInputChannels(); assertEquals(3, channelMap.size()); channelMap @@ -645,13 +619,13 @@ public void testRequestBackoffConfiguration() throws Exception { throw new RuntimeException(e); } }); - InputChannel localChannel = channelMap.get(partitionIds[0]); + InputChannel localChannel = channelMap.get(createSubpartitionInfo(partitionIds[0])); assertEquals(LocalInputChannel.class, localChannel.getClass()); - InputChannel remoteChannel = channelMap.get(partitionIds[1]); + InputChannel remoteChannel = channelMap.get(createSubpartitionInfo(partitionIds[1])); assertEquals(RemoteInputChannel.class, remoteChannel.getClass()); - InputChannel unknownChannel = channelMap.get(partitionIds[2]); + InputChannel unknownChannel = channelMap.get(createSubpartitionInfo(partitionIds[2])); assertEquals(UnknownInputChannel.class, unknownChannel.getClass()); InputChannel[] channels = @@ -749,7 +723,11 @@ public void testRequestBuffersWithUnknownInputChannel() throws Exception { RemoteInputChannel remote = (RemoteInputChannel) - inputGate.getInputChannels().get(resultPartitionId.getPartitionId()); + inputGate + .getInputChannels() + .get( + createSubpartitionInfo( + resultPartitionId.getPartitionId())); // only the exclusive buffers should be assigned/available now assertEquals(buffersPerChannel, remote.getNumberOfAvailableBuffers()); @@ -806,10 +784,14 @@ public void testUpdateUnknownInputChannel() throws Exception { inputGate.setup(); assertThat( - inputGate.getInputChannels().get(remoteResultPartitionId.getPartitionId()), + inputGate + .getInputChannels() + .get(createSubpartitionInfo(remoteResultPartitionId.getPartitionId())), is(instanceOf((UnknownInputChannel.class)))); assertThat( - inputGate.getInputChannels().get(localResultPartitionId.getPartitionId()), + inputGate + .getInputChannels() + .get(createSubpartitionInfo(localResultPartitionId.getPartitionId())), is(instanceOf((UnknownInputChannel.class)))); ResourceID localLocation = ResourceID.generate(); @@ -821,10 +803,14 @@ public void testUpdateUnknownInputChannel() throws Exception { remoteResultPartitionId.getPartitionId(), ResourceID.generate())); assertThat( - inputGate.getInputChannels().get(remoteResultPartitionId.getPartitionId()), + inputGate + .getInputChannels() + .get(createSubpartitionInfo(remoteResultPartitionId.getPartitionId())), is(instanceOf((RemoteInputChannel.class)))); assertThat( - inputGate.getInputChannels().get(localResultPartitionId.getPartitionId()), + inputGate + .getInputChannels() + .get(createSubpartitionInfo(localResultPartitionId.getPartitionId())), is(instanceOf((UnknownInputChannel.class)))); // Trigger updates to local input channel from unknown input channel @@ -834,14 +820,109 @@ public void testUpdateUnknownInputChannel() throws Exception { localResultPartitionId.getPartitionId(), localLocation)); assertThat( - inputGate.getInputChannels().get(remoteResultPartitionId.getPartitionId()), + inputGate + .getInputChannels() + .get(createSubpartitionInfo(remoteResultPartitionId.getPartitionId())), is(instanceOf((RemoteInputChannel.class)))); assertThat( - inputGate.getInputChannels().get(localResultPartitionId.getPartitionId()), + inputGate + .getInputChannels() + .get(createSubpartitionInfo(localResultPartitionId.getPartitionId())), is(instanceOf((LocalInputChannel.class)))); } } + @Test + public void testSingleInputGateWithSubpartitionIndexRange() + throws IOException, InterruptedException { + + IntermediateResultPartitionID[] partitionIds = + new IntermediateResultPartitionID[] { + new IntermediateResultPartitionID(), + new IntermediateResultPartitionID(), + new IntermediateResultPartitionID() + }; + + SubpartitionIndexRange subpartitionIndexRange = new SubpartitionIndexRange(0, 1); + final NettyShuffleEnvironment netEnv = new NettyShuffleEnvironmentBuilder().build(); + + ResourceID localLocation = ResourceID.generate(); + + SingleInputGate gate = + createSingleInputGate( + partitionIds, + ResultPartitionType.BLOCKING, + subpartitionIndexRange, + netEnv, + localLocation, + new TestingConnectionManager(), + new TestingResultPartitionManager(new NoOpResultSubpartitionView())); + + for (InputChannel channel : gate.getInputChannels().values()) { + if (channel instanceof ChannelStateHolder) { + ((ChannelStateHolder) channel).setChannelStateWriter(ChannelStateWriter.NO_OP); + } + } + + SubpartitionInfo info1 = createSubpartitionInfo(partitionIds[0], 0); + SubpartitionInfo info2 = createSubpartitionInfo(partitionIds[0], 1); + SubpartitionInfo info3 = createSubpartitionInfo(partitionIds[1], 0); + SubpartitionInfo info4 = createSubpartitionInfo(partitionIds[1], 1); + SubpartitionInfo info5 = createSubpartitionInfo(partitionIds[2], 0); + SubpartitionInfo info6 = createSubpartitionInfo(partitionIds[2], 1); + + assertThat(gate.getInputChannels().size(), is(6)); + assertThat(gate.getInputChannels().get(info1).getConsumedSubpartitionIndex(), is(0)); + assertThat(gate.getInputChannels().get(info2).getConsumedSubpartitionIndex(), is(1)); + assertThat(gate.getInputChannels().get(info3).getConsumedSubpartitionIndex(), is(0)); + assertThat(gate.getInputChannels().get(info4).getConsumedSubpartitionIndex(), is(1)); + assertThat(gate.getInputChannels().get(info5).getConsumedSubpartitionIndex(), is(0)); + assertThat(gate.getInputChannels().get(info6).getConsumedSubpartitionIndex(), is(1)); + + assertChannelsType(gate, LocalRecoveredInputChannel.class, Arrays.asList(info1, info2)); + assertChannelsType(gate, RemoteRecoveredInputChannel.class, Arrays.asList(info3, info4)); + assertChannelsType(gate, UnknownInputChannel.class, Arrays.asList(info5, info6)); + + // test setup + gate.setup(); + assertNotNull(gate.getBufferPool()); + assertEquals(1, gate.getBufferPool().getNumberOfRequiredMemorySegments()); + + gate.finishReadRecoveredState(); + while (!gate.getStateConsumedFuture().isDone()) { + gate.pollNext(); + } + + // test request partitions + gate.requestPartitions(); + gate.pollNext(); + assertChannelsType(gate, LocalInputChannel.class, Arrays.asList(info1, info2)); + assertChannelsType(gate, RemoteInputChannel.class, Arrays.asList(info3, info4)); + assertChannelsType(gate, UnknownInputChannel.class, Arrays.asList(info5, info6)); + for (InputChannel inputChannel : gate.getInputChannels().values()) { + if (inputChannel instanceof RemoteInputChannel) { + assertNotNull(((RemoteInputChannel) inputChannel).getPartitionRequestClient()); + assertEquals(2, ((RemoteInputChannel) inputChannel).getInitialCredit()); + } else if (inputChannel instanceof LocalInputChannel) { + assertNotNull(((LocalInputChannel) inputChannel).getSubpartitionView()); + } + } + + // test update channels + gate.updateInputChannel( + localLocation, createRemoteWithIdAndLocation(partitionIds[2], localLocation)); + assertChannelsType(gate, LocalInputChannel.class, Arrays.asList(info1, info2)); + assertChannelsType(gate, RemoteInputChannel.class, Arrays.asList(info3, info4)); + assertChannelsType(gate, LocalInputChannel.class, Arrays.asList(info5, info6)); + } + + private void assertChannelsType( + SingleInputGate gate, Class clazz, List infos) { + for (SubpartitionInfo subpartitionInfo : infos) { + assertThat(gate.getInputChannels().get(subpartitionInfo), instanceOf(clazz)); + } + } + @Test public void testQueuedBuffers() throws Exception { final NettyShuffleEnvironment network = createNettyShuffleEnvironment(); @@ -898,7 +979,7 @@ public void testQueuedBuffers() throws Exception { */ @Test public void testPartitionNotFoundExceptionWhileGetNextBuffer() throws Exception { - final SingleInputGate inputGate = createSingleInputGate(1); + final SingleInputGate inputGate = InputChannelTestUtils.createSingleInputGate(1); final LocalInputChannel localChannel = createLocalInputChannel(inputGate, new ResultPartitionManager()); final ResultPartitionID partitionId = localChannel.getPartitionId(); @@ -916,7 +997,7 @@ public void testPartitionNotFoundExceptionWhileGetNextBuffer() throws Exception @Test public void testAnnounceBufferSize() throws Exception { - final SingleInputGate inputGate = createSingleInputGate(2); + final SingleInputGate inputGate = InputChannelTestUtils.createSingleInputGate(2); final LocalInputChannel localChannel = createLocalInputChannel( inputGate, @@ -1058,6 +1139,81 @@ public void testBufferInUseCount() throws Exception { // --------------------------------------------------------------------------------------------- + private static SubpartitionInfo createSubpartitionInfo( + IntermediateResultPartitionID partitionId) { + return createSubpartitionInfo(partitionId, 0); + } + + private static SubpartitionInfo createSubpartitionInfo( + IntermediateResultPartitionID partitionId, int subpartitionIndex) { + return new SubpartitionInfo(partitionId, subpartitionIndex); + } + + static SingleInputGate createSingleInputGate( + IntermediateResultPartitionID[] partitionIds, + ResultPartitionType resultPartitionType, + NettyShuffleEnvironment netEnv) + throws IOException { + return createSingleInputGate( + partitionIds, + resultPartitionType, + new SubpartitionIndexRange(0, 0), + netEnv, + ResourceID.generate(), + null, + null); + } + + static SingleInputGate createSingleInputGate( + IntermediateResultPartitionID[] partitionIds, + ResultPartitionType resultPartitionType, + SubpartitionIndexRange subpartitionIndexRange, + NettyShuffleEnvironment netEnv, + ResourceID localLocation, + ConnectionManager connectionManager, + ResultPartitionManager resultPartitionManager) + throws IOException { + + ShuffleDescriptor[] channelDescs = + new ShuffleDescriptor[] { + // Local + createRemoteWithIdAndLocation(partitionIds[0], localLocation), + // Remote + createRemoteWithIdAndLocation(partitionIds[1], ResourceID.generate()), + // Unknown + new UnknownShuffleDescriptor( + new ResultPartitionID(partitionIds[2], new ExecutionAttemptID())) + }; + + InputGateDeploymentDescriptor gateDesc = + new InputGateDeploymentDescriptor( + new IntermediateDataSetID(), + resultPartitionType, + subpartitionIndexRange, + new TaskDeploymentDescriptor.NonOffloaded<>( + CompressedSerializedValue.fromObject(channelDescs))); + + final TaskMetricGroup taskMetricGroup = + UnregisteredMetricGroups.createUnregisteredTaskMetricGroup(); + return new SingleInputGateFactory( + localLocation, + netEnv.getConfiguration(), + connectionManager != null + ? connectionManager + : netEnv.getConnectionManager(), + resultPartitionManager != null + ? resultPartitionManager + : netEnv.getResultPartitionManager(), + new TaskEventDispatcher(), + netEnv.getNetworkBufferPool()) + .create( + netEnv.createShuffleIOOwnerContext( + "TestTask", taskMetricGroup.executionId(), taskMetricGroup), + 0, + gateDesc, + SingleInputGateBuilder.NO_OP_PRODUCER_CHECKER); + } + private static Map createInputGateWithLocalChannels( NettyShuffleEnvironment network, int numberOfGates, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java index c891a730a2079..006161fee2063 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/TestInputChannel.java @@ -80,6 +80,7 @@ public TestInputChannel( new ResultPartitionID(), 0, 0, + 0, new SimpleCounter(), new SimpleCounter()); this.reuseLastReturnBuffer = reuseLastReturnBuffer; @@ -166,7 +167,7 @@ static TestInputChannel[] createInputChannels( } @Override - void requestSubpartition(int subpartitionIndex) throws IOException, InterruptedException {} + void requestSubpartition() throws IOException, InterruptedException {} @Override Optional getNextBuffer() throws IOException, InterruptedException { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java index 568e937716ae8..ccabb3cf69390 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java @@ -200,7 +200,7 @@ public void testRemoveJobGraph() throws Exception { final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); jobGraphStore.putJobGraph(testingJobGraph); - jobGraphStore.removeJobGraph(testingJobGraph.getJobID()); + jobGraphStore.globalCleanup(testingJobGraph.getJobID()); final JobID actual = removeFuture.get(timeout, TimeUnit.MILLISECONDS); assertThat(actual, is(testingJobGraph.getJobID())); } @@ -213,7 +213,7 @@ public void testRemoveJobGraphWithNonExistName() throws Exception { .build(); final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); - jobGraphStore.removeJobGraph(testingJobGraph.getJobID()); + jobGraphStore.globalCleanup(testingJobGraph.getJobID()); try { removeFuture.get(timeout, TimeUnit.MILLISECONDS); @@ -346,7 +346,7 @@ public void testReleasingJobGraphShouldReleaseHandle() throws Exception { builder.setReleaseConsumer(releaseFuture::complete).build(); final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); jobGraphStore.putJobGraph(testingJobGraph); - jobGraphStore.releaseJobGraph(testingJobGraph.getJobID()); + jobGraphStore.localCleanup(testingJobGraph.getJobID()); final String actual = releaseFuture.get(); assertThat(actual, is(testingJobGraph.getJobID().toString())); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStoreTest.java index 9f89d9f5669f0..d6c85a0c617dc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneJobGraphStoreTest.java @@ -31,7 +31,7 @@ public class StandaloneJobGraphStoreTest { /** Tests that all operations work and don't change the state. */ @Test - public void testNoOps() { + public void testNoOps() throws Exception { StandaloneJobGraphStore jobGraphs = new StandaloneJobGraphStore(); JobGraph jobGraph = JobGraphTestUtils.emptyJobGraph(); @@ -41,7 +41,7 @@ public void testNoOps() { jobGraphs.putJobGraph(jobGraph); assertEquals(0, jobGraphs.getJobIds().size()); - jobGraphs.removeJobGraph(jobGraph.getJobID()); + jobGraphs.globalCleanup(jobGraph.getJobID()); assertEquals(0, jobGraphs.getJobIds().size()); assertNull(jobGraphs.recoverJobGraph(new JobID())); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/TestingJobPersistenceComponentFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/TestingJobPersistenceComponentFactory.java new file mode 100644 index 0000000000000..39fa27bac143d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/TestingJobPersistenceComponentFactory.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.jobmanager; + +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.util.Preconditions; + +/** + * {@code TestingJobPersistenceComponentFactory} implements {@link JobPersistenceComponentFactory} + * for a given {@link JobGraphStore} and {@link JobResultStore}. + */ +public class TestingJobPersistenceComponentFactory implements JobPersistenceComponentFactory { + + private final JobGraphStore jobGraphStore; + private final JobResultStore jobResultStore; + + public TestingJobPersistenceComponentFactory( + JobGraphStore jobGraphStore, JobResultStore jobResultStore) { + this.jobGraphStore = Preconditions.checkNotNull(jobGraphStore); + this.jobResultStore = Preconditions.checkNotNull(jobResultStore); + } + + @Override + public JobGraphStore createJobGraphStore() { + return jobGraphStore; + } + + @Override + public JobResultStore createJobResultStore() { + return jobResultStore; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphsStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphsStoreITCase.java index ad5bcf81829d3..0773c404803a3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphsStoreITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphsStoreITCase.java @@ -130,7 +130,7 @@ public void testPutAndRemoveJobGraph() throws Exception { verifyJobGraphs(jobGraph, jobGraphs.recoverJobGraph(jobId)); // Remove - jobGraphs.removeJobGraph(jobGraph.getJobID()); + jobGraphs.globalCleanup(jobGraph.getJobID()); // Empty state assertEquals(0, jobGraphs.getJobIds().size()); @@ -140,7 +140,7 @@ public void testPutAndRemoveJobGraph() throws Exception { verify(listener, never()).onRemovedJobGraph(any(JobID.class)); // Don't fail if called again - jobGraphs.removeJobGraph(jobGraph.getJobID()); + jobGraphs.globalCleanup(jobGraph.getJobID()); } finally { jobGraphs.stop(); } @@ -193,7 +193,7 @@ public void testRecoverJobGraphs() throws Exception { verifyJobGraphs(expected.get(jobGraph.getJobID()), jobGraph); - jobGraphs.removeJobGraph(jobGraph.getJobID()); + jobGraphs.globalCleanup(jobGraph.getJobID()); } // Empty state @@ -313,7 +313,7 @@ public void testJobGraphRemovalFailureAndLockRelease() throws Exception { assertThat(recoveredJobGraph, is(notNullValue())); try { - otherSubmittedJobGraphStore.removeJobGraph(recoveredJobGraph.getJobID()); + otherSubmittedJobGraphStore.globalCleanup(recoveredJobGraph.getJobID()); fail( "It should not be possible to remove the JobGraph since the first store still has a lock on it."); } catch (Exception ignored) { @@ -323,7 +323,7 @@ public void testJobGraphRemovalFailureAndLockRelease() throws Exception { submittedJobGraphStore.stop(); // now we should be able to delete the job graph - otherSubmittedJobGraphStore.removeJobGraph(recoveredJobGraph.getJobID()); + otherSubmittedJobGraphStore.globalCleanup(recoveredJobGraph.getJobID()); assertThat( otherSubmittedJobGraphStore.recoverJobGraph(recoveredJobGraph.getJobID()), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/DefaultJobMasterServiceProcessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/DefaultJobMasterServiceProcessTest.java index db7faa9fe2a2f..9b9fa8908f0ee 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/DefaultJobMasterServiceProcessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/DefaultJobMasterServiceProcessTest.java @@ -52,7 +52,7 @@ public class DefaultJobMasterServiceProcessTest extends TestLogger { private static final Function failedArchivedExecutionGraphFactory = (throwable -> - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, "test", JobStatus.FAILED, throwable, null, 1337)); @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java index f79893d0c26ec..388264f6cb4d4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterServiceLeadershipRunnerTest.java @@ -23,11 +23,13 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager; import org.apache.flink.runtime.execution.librarycache.TestingClassLoaderLease; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; -import org.apache.flink.runtime.highavailability.RunningJobsRegistry; -import org.apache.flink.runtime.highavailability.nonha.standalone.StandaloneRunningJobsRegistry; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; import org.apache.flink.runtime.jobgraph.JobVertex; @@ -38,6 +40,7 @@ import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; import org.apache.flink.runtime.testtasks.NoOpInvokable; +import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; @@ -88,7 +91,7 @@ public class JobMasterServiceLeadershipRunnerTest extends TestLogger { private TestingFatalErrorHandler fatalErrorHandler; - private RunningJobsRegistry runningJobsRegistry; + private JobResultStore jobResultStore; @BeforeClass public static void setupClass() { @@ -101,7 +104,7 @@ public static void setupClass() { @Before public void setup() { leaderElectionService = new TestingLeaderElectionService(); - runningJobsRegistry = new StandaloneRunningJobsRegistry(); + jobResultStore = new EmbeddedJobResultStore(); fatalErrorHandler = new TestingFatalErrorHandler(); } @@ -256,7 +259,7 @@ public void testJobMasterCreationFailureCompletesJobManagerRunnerWithInitializat @Nonnull private ExecutionGraphInfo createFailedExecutionGraphInfo(FlinkException testException) { return new ExecutionGraphInfo( - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobGraph.getJobID(), jobGraph.getName(), JobStatus.FAILED, @@ -662,15 +665,17 @@ public void testJobMasterServiceProcessCreationFailureIsForwardedToResultFuture( @Test public void testJobAlreadyDone() throws Exception { - JobID jobID = new JobID(); + final JobID jobId = new JobID(); + final JobResult jobResult = + TestingJobResultStore.createJobResult(jobId, ApplicationStatus.UNKNOWN); + jobResultStore.createDirtyResult(new JobResultEntry(jobResult)); try (JobManagerRunner jobManagerRunner = newJobMasterServiceLeadershipRunnerBuilder() .setJobMasterServiceProcessFactory( TestingJobMasterServiceProcessFactory.newBuilder() - .setJobId(jobID) + .setJobId(jobId) .build()) .build()) { - runningJobsRegistry.setJobFinished(jobID); jobManagerRunner.start(); leaderElectionService.isLeader(UUID.randomUUID()); @@ -721,7 +726,7 @@ public JobMasterServiceLeadershipRunner build() { return new JobMasterServiceLeadershipRunner( jobMasterServiceProcessFactory, leaderElectionService, - runningJobsRegistry, + jobResultStore, classLoaderLease, fatalErrorHandler); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java index 0c0994d990dd8..dd049e053719e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestingJobManagerRunner.java @@ -69,7 +69,7 @@ private TestingJobManagerRunner( final ExecutionGraphInfo suspendedExecutionGraphInfo = new ExecutionGraphInfo( - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, "TestJob", JobStatus.SUSPENDED, null, null, 0L), null); terminationFuture.whenComplete( @@ -155,6 +155,10 @@ public void completeTerminationFuture() { terminationFuture.complete(null); } + public void completeTerminationFutureExceptionally(Throwable expectedException) { + terminationFuture.completeExceptionally(expectedException); + } + public CompletableFuture getTerminationFuture() { return terminationFuture; } @@ -166,7 +170,7 @@ public void completeJobMasterGatewayFuture(JobMasterGateway testingJobMasterGate /** {@code Builder} for instantiating {@link TestingJobManagerRunner} instances. */ public static class Builder { - private JobID jobId = null; + private JobID jobId = new JobID(); private boolean blockingTermination = false; private CompletableFuture jobMasterGatewayFuture = new CompletableFuture<>(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactory.java index 379c747d5eeb3..40aa709bf57f2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactory.java @@ -64,7 +64,7 @@ public JobID getJobId() { @Override public ArchivedExecutionGraph createArchivedExecutionGraph( JobStatus jobStatus, @Nullable Throwable cause) { - return ArchivedExecutionGraph.createFromInitializingJob( + return ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, jobName, jobStatus, cause, null, initializationTimestamp); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactoryOld.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactoryOld.java index 986b1bd9be05d..2f316be42b6fd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactoryOld.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/factories/TestingJobMasterServiceProcessFactoryOld.java @@ -72,7 +72,7 @@ public JobID getJobId() { @Override public ArchivedExecutionGraph createArchivedExecutionGraph( JobStatus jobStatus, @Nullable Throwable cause) { - return ArchivedExecutionGraph.createFromInitializingJob( + return ArchivedExecutionGraph.createSparseArchivedExecutionGraph( jobId, "test-job", jobStatus, cause, null, System.currentTimeMillis()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java index 958c01a4a189b..ed74b2e363019 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/metrics/groups/JobManagerGroupTest.java @@ -62,12 +62,12 @@ public void addAndRemoveJobs() throws Exception { assertEquals(2, group.numRegisteredJobMetricGroups()); - group.removeJob(jid1); + group.globalCleanup(jid1); assertTrue(jmJobGroup11.isClosed()); assertEquals(1, group.numRegisteredJobMetricGroups()); - group.removeJob(jid2); + group.globalCleanup(jid2); assertTrue(jmJobGroup21.isClosed()); assertEquals(0, group.numRegisteredJobMetricGroups()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphInfoTest.java index ea5cb9d6a7720..e94aea2f763d2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphInfoTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/ExecutionGraphInfoTest.java @@ -36,7 +36,7 @@ public class ExecutionGraphInfoTest { @Test public void testExecutionGraphHistoryBeingDerivedFromFailedExecutionGraph() { final ArchivedExecutionGraph executionGraph = - ArchivedExecutionGraph.createFromInitializingJob( + ArchivedExecutionGraph.createSparseArchivedExecutionGraph( new JobID(), "test job name", JobStatus.FAILED, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java index 8f5bffcf594ad..641a9181bc8ef 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java @@ -75,7 +75,6 @@ public void testSettingMaxNumberOfCheckpointsToRetain() throws Exception { final CompletedCheckpointStore completedCheckpointStore = SchedulerUtils.createCompletedCheckpointStore( jobManagerConfig, - getClass().getClassLoader(), new StandaloneCheckpointRecoveryFactory(), Executors.directExecutor(), log, @@ -102,7 +101,6 @@ public void testSharedStateRegistration() throws Exception { CompletedCheckpointStore checkpointStore = SchedulerUtils.createCompletedCheckpointStore( new Configuration(), - getClass().getClassLoader(), recoveryFactory, Executors.directExecutor(), log, @@ -123,7 +121,6 @@ private CheckpointRecoveryFactory buildRecoveryFactory(CompletedCheckpoint check public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, - ClassLoader userClassLoader, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor) { List checkpoints = singletonList(checkpoint); 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 4badfbd971254..31fba295d58d7 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 @@ -806,12 +806,15 @@ public void testCloseShutsDownCheckpointingComponents() throws Exception { final CompletableFuture completedCheckpointStoreShutdownFuture = new CompletableFuture<>(); final CompletedCheckpointStore completedCheckpointStore = - new TestingCompletedCheckpointStore(completedCheckpointStoreShutdownFuture); + TestingCompletedCheckpointStore + .createStoreWithShutdownCheckAndNoCompletedCheckpoints( + completedCheckpointStoreShutdownFuture); final CompletableFuture checkpointIdCounterShutdownFuture = new CompletableFuture<>(); final CheckpointIDCounter checkpointIdCounter = - new TestingCheckpointIDCounter(checkpointIdCounterShutdownFuture); + TestingCheckpointIDCounter.createStoreWithShutdownCheckAndNoStartAction( + checkpointIdCounterShutdownFuture); final JobGraph jobGraph = createJobGraph(); // checkpointing components are only created if checkpointing is enabled diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatedTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatedTest.java index 3709149bb649b..6887308f01b9a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatedTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatedTest.java @@ -117,7 +117,7 @@ public void goToFinished(ArchivedExecutionGraph archivedExecutionGraph) { @Override public ArchivedExecutionGraph getArchivedExecutionGraph( JobStatus jobStatus, @Nullable Throwable cause) { - return ArchivedExecutionGraph.createFromInitializingJob( + return ArchivedExecutionGraph.createSparseArchivedExecutionGraph( new JobID(), "testJob", jobStatus, cause, null, 0L); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatingExecutionGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatingExecutionGraphTest.java index 12533aa1c71b1..edf7bbc77e847 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatingExecutionGraphTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/CreatingExecutionGraphTest.java @@ -201,7 +201,7 @@ public void goToExecuting(ExecutionGraph executionGraph) { @Override public ArchivedExecutionGraph getArchivedExecutionGraph( JobStatus jobStatus, @Nullable Throwable cause) { - return ArchivedExecutionGraph.createFromInitializingJob( + return ArchivedExecutionGraph.createSparseArchivedExecutionGraph( new JobID(), "testJob", jobStatus, cause, null, 0L); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java index 4c31f78554699..9fa121a31d39f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobGraphStore.java @@ -54,9 +54,9 @@ public class TestingJobGraphStore implements JobGraphStore { private final ThrowingConsumer putJobGraphConsumer; - private final ThrowingConsumer removeJobGraphConsumer; + private final ThrowingConsumer globalCleanupConsumer; - private final ThrowingConsumer releaseJobGraphConsumer; + private final ThrowingConsumer localCleanupConsumer; private boolean started; @@ -68,16 +68,16 @@ private TestingJobGraphStore( BiFunctionWithException, JobGraph, ? extends Exception> recoverJobGraphFunction, ThrowingConsumer putJobGraphConsumer, - ThrowingConsumer removeJobGraphConsumer, - ThrowingConsumer releaseJobGraphConsumer, + ThrowingConsumer globalCleanupConsumer, + ThrowingConsumer localCleanupConsumer, Collection initialJobGraphs) { this.startConsumer = startConsumer; this.stopRunnable = stopRunnable; this.jobIdsFunction = jobIdsFunction; this.recoverJobGraphFunction = recoverJobGraphFunction; this.putJobGraphConsumer = putJobGraphConsumer; - this.removeJobGraphConsumer = removeJobGraphConsumer; - this.releaseJobGraphConsumer = releaseJobGraphConsumer; + this.globalCleanupConsumer = globalCleanupConsumer; + this.localCleanupConsumer = localCleanupConsumer; for (JobGraph initialJobGraph : initialJobGraphs) { storedJobs.put(initialJobGraph.getJobID(), initialJobGraph); @@ -110,16 +110,16 @@ public synchronized void putJobGraph(JobGraph jobGraph) throws Exception { } @Override - public synchronized void removeJobGraph(JobID jobId) throws Exception { + public synchronized void globalCleanup(JobID jobId) throws Exception { verifyIsStarted(); - removeJobGraphConsumer.accept(jobId); + globalCleanupConsumer.accept(jobId); storedJobs.remove(jobId); } @Override - public synchronized void releaseJobGraph(JobID jobId) throws Exception { + public synchronized void localCleanup(JobID jobId) throws Exception { verifyIsStarted(); - releaseJobGraphConsumer.accept(jobId); + localCleanupConsumer.accept(jobId); } @Override @@ -141,6 +141,7 @@ public static Builder newBuilder() { return new Builder(); } + /** {@code Builder} for creating {@code TestingJobGraphStore} instances. */ public static class Builder { private ThrowingConsumer startConsumer = ignored -> {}; @@ -155,10 +156,9 @@ public static class Builder { private ThrowingConsumer putJobGraphConsumer = ignored -> {}; - private ThrowingConsumer removeJobGraphConsumer = ignored -> {}; + private ThrowingConsumer globalCleanupConsumer = ignored -> {}; - private ThrowingConsumer releaseJobGraphConsumer = - ignored -> {}; + private ThrowingConsumer localCleanupConsumer = ignored -> {}; private Collection initialJobGraphs = Collections.emptyList(); @@ -197,15 +197,15 @@ public Builder setPutJobGraphConsumer( return this; } - public Builder setRemoveJobGraphConsumer( - ThrowingConsumer removeJobGraphConsumer) { - this.removeJobGraphConsumer = removeJobGraphConsumer; + public Builder setGlobalCleanupConsumer( + ThrowingConsumer globalCleanupConsumer) { + this.globalCleanupConsumer = globalCleanupConsumer; return this; } - public Builder setReleaseJobGraphConsumer( - ThrowingConsumer releaseJobGraphConsumer) { - this.releaseJobGraphConsumer = releaseJobGraphConsumer; + public Builder setLocalCleanupConsumer( + ThrowingConsumer localCleanupConsumer) { + this.localCleanupConsumer = localCleanupConsumer; return this; } @@ -227,8 +227,8 @@ public TestingJobGraphStore build() { jobIdsFunction, recoverJobGraphFunction, putJobGraphConsumer, - removeJobGraphConsumer, - releaseJobGraphConsumer, + globalCleanupConsumer, + localCleanupConsumer, initialJobGraphs); if (startJobGraphStore) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobResultStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobResultStore.java new file mode 100644 index 0000000000000..a73759196f49d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingJobResultStore.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.testutils; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.highavailability.JobResultEntry; +import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.util.function.FunctionWithException; +import org.apache.flink.util.function.SupplierWithException; +import org.apache.flink.util.function.ThrowingConsumer; + +import java.io.IOException; +import java.util.Collections; +import java.util.Set; + +/** + * {@code TestingJobResultStore} is a {@link JobResultStore} implementation that can be used in + * tests. + */ +public class TestingJobResultStore implements JobResultStore { + + public static final JobResult DUMMY_JOB_RESULT = createSuccessfulJobResult(new JobID()); + + public static JobResult createSuccessfulJobResult(JobID jobId) { + return createJobResult(jobId, ApplicationStatus.SUCCEEDED); + } + + public static JobResult createJobResult(JobID jobId, ApplicationStatus applicationStatus) { + return new JobResult.Builder() + .jobId(jobId) + .applicationStatus(applicationStatus) + .netRuntime(1) + .build(); + } + + private final ThrowingConsumer createDirtyResultConsumer; + private final ThrowingConsumer markResultAsCleanConsumer; + private final FunctionWithException + hasJobResultEntryFunction; + private final FunctionWithException + hasDirtyJobResultEntryFunction; + private final FunctionWithException + hasCleanJobResultEntryFunction; + private final SupplierWithException, ? extends IOException> + getDirtyResultsSupplier; + + private TestingJobResultStore( + ThrowingConsumer createDirtyResultConsumer, + ThrowingConsumer markResultAsCleanConsumer, + FunctionWithException hasJobResultEntryFunction, + FunctionWithException + hasDirtyJobResultEntryFunction, + FunctionWithException + hasCleanJobResultEntryFunction, + SupplierWithException, ? extends IOException> getDirtyResultsSupplier) { + this.createDirtyResultConsumer = createDirtyResultConsumer; + this.markResultAsCleanConsumer = markResultAsCleanConsumer; + this.hasJobResultEntryFunction = hasJobResultEntryFunction; + this.hasDirtyJobResultEntryFunction = hasDirtyJobResultEntryFunction; + this.hasCleanJobResultEntryFunction = hasCleanJobResultEntryFunction; + this.getDirtyResultsSupplier = getDirtyResultsSupplier; + } + + @Override + public void createDirtyResult(JobResultEntry jobResultEntry) throws IOException { + createDirtyResultConsumer.accept(jobResultEntry); + } + + @Override + public void markResultAsClean(JobID jobId) throws IOException { + markResultAsCleanConsumer.accept(jobId); + } + + @Override + public boolean hasJobResultEntry(JobID jobId) throws IOException { + return hasJobResultEntryFunction.apply(jobId); + } + + @Override + public boolean hasDirtyJobResultEntry(JobID jobId) throws IOException { + return hasDirtyJobResultEntryFunction.apply(jobId); + } + + @Override + public boolean hasCleanJobResultEntry(JobID jobId) throws IOException { + return hasCleanJobResultEntryFunction.apply(jobId); + } + + @Override + public Set getDirtyResults() throws IOException { + return getDirtyResultsSupplier.get(); + } + + public static TestingJobResultStore.Builder builder() { + return new Builder(); + } + + /** {@code Builder} for instantiating {@code TestingJobResultStore} instances. */ + public static class Builder { + + private ThrowingConsumer createDirtyResultConsumer = + ignored -> {}; + private ThrowingConsumer markResultAsCleanConsumer = + ignored -> {}; + + private FunctionWithException + hasJobResultEntryFunction = ignored -> false; + private FunctionWithException + hasDirtyJobResultEntryFunction = ignored -> false; + private FunctionWithException + hasCleanJobResultEntryFunction = ignored -> false; + + private SupplierWithException, ? extends IOException> + getDirtyResultsSupplier = Collections::emptySet; + + public Builder withCreateDirtyResultConsumer( + ThrowingConsumer createDirtyResultConsumer) { + this.createDirtyResultConsumer = createDirtyResultConsumer; + return this; + } + + public Builder withMarkResultAsCleanConsumer( + ThrowingConsumer markResultAsCleanConsumer) { + this.markResultAsCleanConsumer = markResultAsCleanConsumer; + return this; + } + + public Builder withHasJobResultEntryFunction( + FunctionWithException + hasJobResultEntryFunction) { + this.hasJobResultEntryFunction = hasJobResultEntryFunction; + return this; + } + + public Builder withHasDirtyJobResultEntryFunction( + FunctionWithException + hasDirtyJobResultEntryFunction) { + this.hasDirtyJobResultEntryFunction = hasDirtyJobResultEntryFunction; + return this; + } + + public Builder withHasCleanJobResultEntryFunction( + FunctionWithException + hasCleanJobResultEntryFunction) { + this.hasCleanJobResultEntryFunction = hasCleanJobResultEntryFunction; + return this; + } + + public Builder withGetDirtyResultsSupplier( + SupplierWithException, ? extends IOException> + getDirtyResultsSupplier) { + this.getDirtyResultsSupplier = getDirtyResultsSupplier; + return this; + } + + public TestingJobResultStore build() { + return new TestingJobResultStore( + createDirtyResultConsumer, + markResultAsCleanConsumer, + hasJobResultEntryFunction, + hasDirtyJobResultEntryFunction, + hasCleanJobResultEntryFunction, + getDirtyResultsSupplier); + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/SingleInputGateBenchmarkFactory.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/SingleInputGateBenchmarkFactory.java index b1276b2786d85..7dda72c3f4963 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/SingleInputGateBenchmarkFactory.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/SingleInputGateBenchmarkFactory.java @@ -64,6 +64,7 @@ protected InputChannel createKnownInputChannel( SingleInputGate inputGate, int index, NettyShuffleDescriptor inputChannelDescriptor, + int consumedSubpartitionIndex, SingleInputGateFactory.ChannelStatistics channelStatistics, InputChannelMetrics metrics) { ResultPartitionID partitionId = inputChannelDescriptor.getResultPartitionID(); @@ -72,6 +73,7 @@ protected InputChannel createKnownInputChannel( inputGate, index, partitionId, + index, partitionManager, taskEventPublisher, partitionRequestInitialBackoff, @@ -82,6 +84,7 @@ protected InputChannel createKnownInputChannel( inputGate, index, partitionId, + index, inputChannelDescriptor.getConnectionId(), connectionManager, partitionRequestInitialBackoff, @@ -103,6 +106,7 @@ public TestLocalInputChannel( SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, ResultPartitionManager partitionManager, TaskEventPublisher taskEventPublisher, int initialBackoff, @@ -112,6 +116,7 @@ public TestLocalInputChannel( inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, partitionManager, taskEventPublisher, initialBackoff, @@ -122,8 +127,8 @@ public TestLocalInputChannel( } @Override - public void requestSubpartition(int subpartitionIndex) throws IOException { - super.requestSubpartition(getChannelIndex()); + public void requestSubpartition() throws IOException { + super.requestSubpartition(); } @Override @@ -152,6 +157,7 @@ public TestRemoteInputChannel( SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, + int consumedSubpartitionIndex, ConnectionID connectionId, ConnectionManager connectionManager, int initialBackOff, @@ -162,6 +168,7 @@ public TestRemoteInputChannel( inputGate, channelIndex, partitionId, + consumedSubpartitionIndex, connectionId, connectionManager, initialBackOff, @@ -173,9 +180,8 @@ public TestRemoteInputChannel( } @Override - public void requestSubpartition(int subpartitionIndex) - throws IOException, InterruptedException { - super.requestSubpartition(getChannelIndex()); + public void requestSubpartition() throws IOException, InterruptedException { + super.requestSubpartition(); } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/checkpointing/CheckpointedInputGateTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/checkpointing/CheckpointedInputGateTest.java index ad3857f36b593..b49c47d292759 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/checkpointing/CheckpointedInputGateTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/checkpointing/CheckpointedInputGateTest.java @@ -218,7 +218,7 @@ public void testPriorityBeforeClose() throws IOException, InterruptedException { .setChannelFactory(InputChannelBuilder::buildRemoteChannel) .build(); singleInputGate.setup(); - ((RemoteInputChannel) singleInputGate.getChannel(0)).requestSubpartition(0); + ((RemoteInputChannel) singleInputGate.getChannel(0)).requestSubpartition(); final TaskMailboxImpl mailbox = new TaskMailboxImpl(); MailboxExecutorImpl mailboxExecutor = @@ -355,7 +355,7 @@ public void invoke() {} mailboxExecutor, UpstreamRecoveryTracker.forInputGate(singleInputGate)); for (int i = 0; i < numberOfChannels; i++) { - ((RemoteInputChannel) checkpointedInputGate.getChannel(i)).requestSubpartition(0); + ((RemoteInputChannel) checkpointedInputGate.getChannel(i)).requestSubpartition(); } return checkpointedInputGate; } @@ -395,7 +395,7 @@ private CheckpointedInputGate setupInputGateWithAlternatingController( mailboxExecutor, UpstreamRecoveryTracker.forInputGate(singleInputGate)); for (int i = 0; i < numberOfChannels; i++) { - ((RemoteInputChannel) checkpointedInputGate.getChannel(i)).requestSubpartition(0); + ((RemoteInputChannel) checkpointedInputGate.getChannel(i)).requestSubpartition(); } return checkpointedInputGate; } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java index 7bda9f3fbcfaf..71089c589dda6 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java @@ -92,10 +92,7 @@ protected void createTestValuesSourceTable( @Nullable String partitionFields, Map extraProperties) { checkArgument(fieldNameAndTypes.length > 0); - String partitionedBy = - StringUtils.isNullOrWhitespaceOnly(partitionFields) - ? "" - : "\n partitioned by (" + partitionFields + ") \n"; + String dataId = TestValuesTableFactory.registerData(data); Map properties = new HashMap<>(); properties.put("connector", "values"); @@ -103,6 +100,19 @@ protected void createTestValuesSourceTable( properties.put("bounded", "true"); properties.put("disable-lookup", "true"); properties.putAll(extraProperties); + createTestSourceTable(tableName, fieldNameAndTypes, partitionFields, properties); + } + + protected void createTestSourceTable( + String tableName, + String[] fieldNameAndTypes, + @Nullable String partitionFields, + Map properties) { + checkArgument(fieldNameAndTypes.length > 0); + String partitionedBy = + StringUtils.isNullOrWhitespaceOnly(partitionFields) + ? "" + : "\n partitioned by (" + partitionFields + ") \n"; String ddl = String.format( "CREATE TABLE %s (\n" + "%s\n" + ") %s with (\n%s)", @@ -140,15 +150,26 @@ protected void createTestValuesSinkTable( String tableName, String[] fieldNameAndTypes, @Nullable String partitionFields, - Map extraProperties) { + Map properties) { + + Map extraProperties = new HashMap<>(); + extraProperties.put("connector", "values"); + + properties.putAll(extraProperties); + + createTestSinkTable(tableName, fieldNameAndTypes, partitionFields, properties); + } + + protected void createTestSinkTable( + String tableName, + String[] fieldNameAndTypes, + @Nullable String partitionFields, + Map properties) { checkArgument(fieldNameAndTypes.length > 0); String partitionedBy = StringUtils.isNullOrWhitespaceOnly(partitionFields) ? "" : "\n partitioned by (" + partitionFields + ") \n"; - Map properties = new HashMap<>(); - properties.put("connector", "values"); - properties.putAll(extraProperties); String ddl = String.format( "CREATE TABLE %s (\n" + "%s\n" + ") %s with (\n%s)",