diff --git a/docs/layouts/shortcodes/generated/application_result_store_configuration.html b/docs/layouts/shortcodes/generated/application_result_store_configuration.html new file mode 100644 index 0000000000000..f910e569bde2b --- /dev/null +++ b/docs/layouts/shortcodes/generated/application_result_store_configuration.html @@ -0,0 +1,24 @@ + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
application-result-store.delete-on-commit
trueBooleanDetermines whether application results should be automatically removed from the underlying application result store when the corresponding entity transitions into a clean state. If false, the cleaned application results are, instead, marked as clean to indicate their state. In this case, Flink no longer has ownership and the resources need to be cleaned up by the user.
application-result-store.storage-path
(none)StringDefines where application results should be stored. This should be an underlying file-system that provides read-after-write consistency. By default, this is {high-availability.storageDir}/application-result-store/{high-availability.cluster-id}.
diff --git a/docs/layouts/shortcodes/generated/common_high_availability_ars_section.html b/docs/layouts/shortcodes/generated/common_high_availability_ars_section.html new file mode 100644 index 0000000000000..f910e569bde2b --- /dev/null +++ b/docs/layouts/shortcodes/generated/common_high_availability_ars_section.html @@ -0,0 +1,24 @@ + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
application-result-store.delete-on-commit
trueBooleanDetermines whether application results should be automatically removed from the underlying application result store when the corresponding entity transitions into a clean state. If false, the cleaned application results are, instead, marked as clean to indicate their state. In this case, Flink no longer has ownership and the resources need to be cleaned up by the user.
application-result-store.storage-path
(none)StringDefines where application results should be stored. This should be an underlying file-system that provides read-after-write consistency. By default, this is {high-availability.storageDir}/application-result-store/{high-availability.cluster-id}.
diff --git a/docs/layouts/shortcodes/generated/common_high_availability_jrs_section.html b/docs/layouts/shortcodes/generated/common_high_availability_jrs_section.html index e6d58ee3ffbfd..1d677d17de1a6 100644 --- a/docs/layouts/shortcodes/generated/common_high_availability_jrs_section.html +++ b/docs/layouts/shortcodes/generated/common_high_availability_jrs_section.html @@ -8,12 +8,6 @@ - -
job-result-store.delete-on-commit
- true - Boolean - Determines whether job results should be automatically removed from the underlying job result store when the corresponding entity transitions into a clean state. If false, the cleaned job results are, instead, marked as clean to indicate their state. In this case, Flink no longer has ownership and the resources need to be cleaned up by the user. -
job-result-store.storage-path
(none) diff --git a/docs/layouts/shortcodes/generated/expert_high_availability_zk_section.html b/docs/layouts/shortcodes/generated/expert_high_availability_zk_section.html index c7ce745452d63..4261b5448704c 100644 --- a/docs/layouts/shortcodes/generated/expert_high_availability_zk_section.html +++ b/docs/layouts/shortcodes/generated/expert_high_availability_zk_section.html @@ -56,6 +56,12 @@ Boolean Defines whether a suspended ZooKeeper connection will be treated as an error that causes the leader information to be invalidated or not. In case you set this option to true, Flink will wait until a ZooKeeper connection is marked as lost before it revokes the leadership of components. This has the effect that Flink is more resilient against temporary connection instabilities at the cost of running more likely into timing issues with ZooKeeper. + +
high-availability.zookeeper.path.applications
+ "/applications" + String + ZooKeeper root path (ZNode) for applications +
high-availability.zookeeper.path.execution-plans
"/execution-plans" diff --git a/docs/layouts/shortcodes/generated/high_availability_configuration.html b/docs/layouts/shortcodes/generated/high_availability_configuration.html index 26a6b300907a2..116086bd88d5c 100644 --- a/docs/layouts/shortcodes/generated/high_availability_configuration.html +++ b/docs/layouts/shortcodes/generated/high_availability_configuration.html @@ -98,6 +98,12 @@ Boolean Defines whether a suspended ZooKeeper connection will be treated as an error that causes the leader information to be invalidated or not. In case you set this option to true, Flink will wait until a ZooKeeper connection is marked as lost before it revokes the leadership of components. This has the effect that Flink is more resilient against temporary connection instabilities at the cost of running more likely into timing issues with ZooKeeper. + +
high-availability.zookeeper.path.applications
+ "/applications" + String + ZooKeeper root path (ZNode) for applications +
high-availability.zookeeper.path.execution-plans
"/execution-plans" diff --git a/docs/layouts/shortcodes/generated/job_result_store_configuration.html b/docs/layouts/shortcodes/generated/job_result_store_configuration.html index e6d58ee3ffbfd..1d677d17de1a6 100644 --- a/docs/layouts/shortcodes/generated/job_result_store_configuration.html +++ b/docs/layouts/shortcodes/generated/job_result_store_configuration.html @@ -8,12 +8,6 @@ - -
job-result-store.delete-on-commit
- true - Boolean - Determines whether job results should be automatically removed from the underlying job result store when the corresponding entity transitions into a clean state. If false, the cleaned job results are, instead, marked as clean to indicate their state. In this case, Flink no longer has ownership and the resources need to be cleaned up by the user. -
job-result-store.storage-path
(none) diff --git a/flink-annotations/src/main/java/org/apache/flink/annotation/docs/Documentation.java b/flink-annotations/src/main/java/org/apache/flink/annotation/docs/Documentation.java index 8b298eac03887..b6f9d1d3ae8e9 100644 --- a/flink-annotations/src/main/java/org/apache/flink/annotation/docs/Documentation.java +++ b/flink-annotations/src/main/java/org/apache/flink/annotation/docs/Documentation.java @@ -68,6 +68,8 @@ public static final class Sections { "common_high_availability_zk"; public static final String COMMON_HIGH_AVAILABILITY_JOB_RESULT_STORE = "common_high_availability_jrs"; + public static final String COMMON_HIGH_AVAILABILITY_APPLICATION_RESULT_STORE = + "common_high_availability_ars"; public static final String COMMON_MEMORY = "common_memory"; public static final String COMMON_MISCELLANEOUS = "common_miscellaneous"; 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 fa7f597863be3..579845cfeebfd 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,15 +26,19 @@ import org.apache.flink.configuration.ApplicationOptionsInternal; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DeploymentOptions; +import org.apache.flink.runtime.application.AbstractApplication; import org.apache.flink.runtime.dispatcher.ApplicationBootstrap; import org.apache.flink.runtime.dispatcher.Dispatcher; 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.PartialDispatcherServicesWithJobPersistenceComponents; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithPersistenceComponents; import org.apache.flink.runtime.dispatcher.runner.AbstractDispatcherLeaderProcess; import org.apache.flink.runtime.dispatcher.runner.DefaultDispatcherGatewayService; +import org.apache.flink.runtime.highavailability.ApplicationResult; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.jobmanager.ApplicationStore; import org.apache.flink.runtime.jobmanager.ExecutionPlanWriter; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; @@ -90,8 +94,12 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( DispatcherId fencingToken, Collection recoveredJobs, Collection recoveredDirtyJobResults, + Collection recoveredApplications, + Collection recoveredDirtyApplicationResults, ExecutionPlanWriter executionPlanWriter, - JobResultStore jobResultStore) { + JobResultStore jobResultStore, + ApplicationStore applicationStore, + ApplicationResultStore applicationResultStore) { final List recoveredJobInfos = getRecoveredJobInfos(recoveredJobs); final List recoveredTerminalJobInfos = @@ -126,12 +134,16 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( fencingToken, recoveredJobs, recoveredDirtyJobResults, + recoveredApplications, + recoveredDirtyApplicationResults, (dispatcherGateway, scheduledExecutor, errorHandler) -> new ApplicationBootstrap(bootstrapApplication), - PartialDispatcherServicesWithJobPersistenceComponents.from( + PartialDispatcherServicesWithPersistenceComponents.from( partialDispatcherServices, executionPlanWriter, - jobResultStore)); + jobResultStore, + applicationStore, + applicationResultStore)); } 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 03b94a88ef9e2..0bf11a5d8aa59 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.JobPersistenceComponentFactory; +import org.apache.flink.runtime.jobmanager.PersistenceComponentFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -59,7 +59,7 @@ private ApplicationDispatcherLeaderProcessFactoryFactory( @Override public DispatcherLeaderProcessFactory createFactory( - JobPersistenceComponentFactory jobPersistenceComponentFactory, + PersistenceComponentFactory persistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices, @@ -75,7 +75,8 @@ public DispatcherLeaderProcessFactory createFactory( return new SessionDispatcherLeaderProcessFactory( dispatcherServiceFactory, - jobPersistenceComponentFactory, + persistenceComponentFactory, + partialDispatcherServices.getBlobServer(), ioExecutor, fatalErrorHandler); } diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/PackagedProgramApplication.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/PackagedProgramApplication.java index 9eb6844798915..6f42ea719b170 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/PackagedProgramApplication.java +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/PackagedProgramApplication.java @@ -34,9 +34,11 @@ import org.apache.flink.configuration.PipelineOptionsInternal; import org.apache.flink.core.execution.PipelineExecutorServiceLoader; import org.apache.flink.runtime.application.AbstractApplication; +import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.jobmanager.ApplicationStoreEntry; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; @@ -48,6 +50,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.time.Duration; import java.util.ArrayList; import java.util.Collection; @@ -95,6 +99,8 @@ public class PackagedProgramApplication extends AbstractApplication { private final boolean shutDownOnFinish; + @Nullable private final PermanentBlobKey userJarBlobKey; + private transient PackagedProgram program; private transient CompletableFuture applicationCompletionFuture; @@ -125,6 +131,28 @@ public PackagedProgramApplication( shutDownOnFinish); } + public PackagedProgramApplication( + final ApplicationID applicationId, + final PackagedProgram program, + final Configuration configuration, + final boolean handleFatalError, + final boolean enforceSingleJobExecution, + final boolean submitFailedJobOnApplicationError, + final boolean shutDownOnFinish, + final @Nullable PermanentBlobKey userJarBlobKey) { + this( + applicationId, + program, + Collections.emptyList(), + Collections.emptyList(), + configuration, + handleFatalError, + enforceSingleJobExecution, + submitFailedJobOnApplicationError, + shutDownOnFinish, + userJarBlobKey); + } + public PackagedProgramApplication( final ApplicationID applicationId, final PackagedProgram program, @@ -135,6 +163,30 @@ public PackagedProgramApplication( final boolean enforceSingleJobExecution, final boolean submitFailedJobOnApplicationError, final boolean shutDownOnFinish) { + this( + applicationId, + program, + recoveredJobInfos, + recoveredTerminalJobInfos, + configuration, + handleFatalError, + enforceSingleJobExecution, + submitFailedJobOnApplicationError, + shutDownOnFinish, + null); + } + + public PackagedProgramApplication( + final ApplicationID applicationId, + final PackagedProgram program, + final Collection recoveredJobInfos, + final Collection recoveredTerminalJobInfos, + final Configuration configuration, + final boolean handleFatalError, + final boolean enforceSingleJobExecution, + final boolean submitFailedJobOnApplicationError, + final boolean shutDownOnFinish, + final @Nullable PermanentBlobKey userJarBlobKey) { super(applicationId); this.program = checkNotNull(program); this.recoveredJobInfos = checkNotNull(recoveredJobInfos); @@ -144,6 +196,7 @@ public PackagedProgramApplication( this.enforceSingleJobExecution = enforceSingleJobExecution; this.submitFailedJobOnApplicationError = submitFailedJobOnApplicationError; this.shutDownOnFinish = shutDownOnFinish; + this.userJarBlobKey = userJarBlobKey; this.programDescriptor = program.getDescriptor(); } @@ -320,6 +373,40 @@ public String getName() { return programDescriptor.getMainClassName(); } + @Override + public Optional getApplicationStoreEntry() { + if (userJarBlobKey == null) { + return Optional.empty(); + } + return Optional.of( + new PackagedProgramApplicationEntry( + configuration, + userJarBlobKey, + programDescriptor.getMainClassName(), + programDescriptor.getProgramArgs(), + getApplicationId(), + getName(), + handleFatalError, + enforceSingleJobExecution, + submitFailedJobOnApplicationError, + shutDownOnFinish)); + } + + @VisibleForTesting + PermanentBlobKey getUserJarBlobKey() { + return userJarBlobKey; + } + + @VisibleForTesting + Collection getRecoveredJobInfos() { + return recoveredJobInfos; + } + + @VisibleForTesting + Collection getRecoveredTerminalJobInfos() { + return recoveredTerminalJobInfos; + } + @VisibleForTesting ScheduledFuture getApplicationExecutionFuture() { return applicationExecutionTask; diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/PackagedProgramApplicationEntry.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/PackagedProgramApplicationEntry.java new file mode 100644 index 0000000000000..6fa042063bc8d --- /dev/null +++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/PackagedProgramApplicationEntry.java @@ -0,0 +1,151 @@ +/* + * 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.deployment.application; + +import org.apache.flink.api.common.ApplicationID; +import org.apache.flink.api.common.JobInfo; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.configuration.ConfigUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.runtime.application.AbstractApplication; +import org.apache.flink.runtime.blob.PermanentBlobKey; +import org.apache.flink.runtime.blob.PermanentBlobService; +import org.apache.flink.runtime.jobmanager.ApplicationStoreEntry; + +import java.io.File; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Collection; +import java.util.List; + +/** {@code ApplicationStoreEntry} for {@code PackagedProgramApplication}. */ +public class PackagedProgramApplicationEntry implements ApplicationStoreEntry { + + private final Configuration configuration; + + private final PermanentBlobKey userJarBlobKey; + + private final String entryClass; + + private final String[] programArgs; + + private final ApplicationID applicationId; + + private final String applicationName; + + private final boolean handleFatalError; + + private final boolean enforceSingleJobExecution; + + private final boolean submitFailedJobOnApplicationError; + + private final boolean shutDownOnFinish; + + public PackagedProgramApplicationEntry( + Configuration configuration, + PermanentBlobKey userJarBlobKey, + String entryClass, + String[] programArgs, + ApplicationID applicationId, + String applicationName, + boolean handleFatalError, + boolean enforceSingleJobExecution, + boolean submitFailedJobOnApplicationError, + boolean shutDownOnFinish) { + this.configuration = configuration; + this.userJarBlobKey = userJarBlobKey; + this.entryClass = entryClass; + this.programArgs = programArgs; + this.applicationId = applicationId; + this.applicationName = applicationName; + this.handleFatalError = handleFatalError; + this.enforceSingleJobExecution = enforceSingleJobExecution; + this.submitFailedJobOnApplicationError = submitFailedJobOnApplicationError; + this.shutDownOnFinish = shutDownOnFinish; + } + + @Override + public AbstractApplication getApplication( + PermanentBlobService blobService, + Collection recoveredJobInfos, + Collection recoveredTerminalJobInfos) { + File jarFile; + try { + jarFile = blobService.getFile(applicationId, userJarBlobKey); + } catch (Exception e) { + throw new RuntimeException("Failed to get user jar file from blob", e); + } + + if (!jarFile.exists()) { + throw new RuntimeException(String.format("Jar file %s does not exist", jarFile)); + } + + PackagedProgram program; + try { + program = + PackagedProgram.newBuilder() + .setJarFile(jarFile) + .setEntryPointClassName(entryClass) + .setConfiguration(configuration) + .setUserClassPaths(getClasspaths()) + .setArguments(programArgs) + .build(); + } catch (Exception e) { + throw new RuntimeException( + String.format( + "Failed to create PackagedProgram for application %s", applicationId)); + } + + return new PackagedProgramApplication( + applicationId, + program, + recoveredJobInfos, + recoveredTerminalJobInfos, + configuration, + handleFatalError, + enforceSingleJobExecution, + submitFailedJobOnApplicationError, + shutDownOnFinish, + userJarBlobKey); + } + + private List getClasspaths() { + try { + return ConfigUtils.decodeListFromConfig( + configuration, PipelineOptions.CLASSPATHS, URL::new); + } catch (MalformedURLException e) { + throw new RuntimeException( + String.format( + "Failed to extract '%s' as URLs. Provided value: %s", + PipelineOptions.CLASSPATHS.key(), + configuration.get(PipelineOptions.CLASSPATHS))); + } + } + + @Override + public ApplicationID getApplicationId() { + return applicationId; + } + + @Override + public String getName() { + return applicationName; + } +} diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramDescriptor.java b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramDescriptor.java index d36a16a8c698d..0983f14da1415 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramDescriptor.java +++ b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramDescriptor.java @@ -70,6 +70,10 @@ public String getMainClassName() { return mainClassName; } + public String[] getProgramArgs() { + return programArgs; + } + public PackagedProgram toPackageProgram() throws ProgramInvocationException { return PackagedProgram.newBuilder() .setJarFile(jarFile) diff --git a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/PackagedProgramApplicationEntryTest.java b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/PackagedProgramApplicationEntryTest.java new file mode 100644 index 0000000000000..d6e559d6ad75e --- /dev/null +++ b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/PackagedProgramApplicationEntryTest.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 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.deployment.application; + +import org.apache.flink.api.common.ApplicationID; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobInfo; +import org.apache.flink.api.common.JobInfoImpl; +import org.apache.flink.client.cli.CliFrontendTestUtils; +import org.apache.flink.client.program.PackagedProgram; +import org.apache.flink.client.testjar.MultiExecuteJob; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.application.AbstractApplication; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.PermanentBlobKey; +import org.apache.flink.runtime.blob.VoidBlobStore; +import org.apache.flink.runtime.jobmanager.ApplicationStoreEntry; +import org.apache.flink.testutils.junit.utils.TempDirUtils; + +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.FileInputStream; +import java.io.InputStream; +import java.nio.file.Path; +import java.util.Collections; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertNull; + +/** Tests for the {@link PackagedProgramApplicationEntry}. */ +class PackagedProgramApplicationEntryTest { + + @TempDir private static Path temporaryFolder; + + private static BlobServer blobServer; + private static Configuration config; + private static ApplicationID applicationId; + private static String applicationName; + private static File jarFile; + private static String entryClass; + private static String[] programArgs; + private static PackagedProgram program; + + @BeforeAll + static void setup() throws Exception { + config = new Configuration(); + blobServer = + new BlobServer( + config, TempDirUtils.newFolder(temporaryFolder), new VoidBlobStore()); + blobServer.start(); + + applicationId = new ApplicationID(); + applicationName = MultiExecuteJob.class.getCanonicalName(); + entryClass = MultiExecuteJob.class.getName(); + programArgs = new String[] {"1", "true"}; + jarFile = new File(CliFrontendTestUtils.getTestJarPath()); + program = + PackagedProgram.newBuilder() + .setJarFile(jarFile) + .setEntryPointClassName(entryClass) + .setConfiguration(config) + .setArguments(programArgs) + .build(); + } + + @Test + void testConstructionWithoutJarBlob() { + PackagedProgramApplication application = + new PackagedProgramApplication( + applicationId, program, config, true, false, false, true); + ApplicationStoreEntry entry = application.getApplicationStoreEntry().orElse(null); + + assertNull(entry); + } + + @Test + void testConstruction() throws Exception { + PermanentBlobKey blobKey; + try (InputStream is = new FileInputStream(jarFile)) { + blobKey = blobServer.putPermanent(applicationId, is); + } + + PackagedProgramApplication application = + new PackagedProgramApplication( + applicationId, program, config, true, false, false, true, blobKey); + ApplicationStoreEntry entry = application.getApplicationStoreEntry().orElse(null); + + assertInstanceOf(PackagedProgramApplicationEntry.class, entry); + assertEquals(applicationId, entry.getApplicationId()); + assertEquals(applicationName, entry.getName()); + + JobInfo recoveredJob = new JobInfoImpl(new JobID(), "test"); + JobInfo recoveredTerminalJob = new JobInfoImpl(new JobID(), "test"); + AbstractApplication reconstructed = + entry.getApplication( + blobServer, + Collections.singleton(recoveredJob), + Collections.singleton(recoveredTerminalJob)); + + assertInstanceOf(PackagedProgramApplication.class, reconstructed); + + PackagedProgramApplication packagedProgramApplication = + (PackagedProgramApplication) reconstructed; + + assertEquals(blobKey, packagedProgramApplication.getUserJarBlobKey()); + assertThat( + packagedProgramApplication.getRecoveredJobInfos().stream() + .map(JobInfo::getJobId) + .collect(Collectors.toList())) + .containsExactly(recoveredJob.getJobId()); + assertThat( + packagedProgramApplication.getRecoveredTerminalJobInfos().stream() + .map(JobInfo::getJobId) + .collect(Collectors.toList())) + .containsExactly(recoveredTerminalJob.getJobId()); + } + + @Test + void testConstructionFailsWithInvalidJar() throws Exception { + PermanentBlobKey blobKey = blobServer.putPermanent(applicationId, new byte[] {1, 2, 3}); + + PackagedProgramApplicationEntry entry = createEntryWithJarBlob(blobKey); + + assertThatThrownBy( + () -> + entry.getApplication( + blobServer, + Collections.emptyList(), + Collections.emptyList())) + .hasMessageContaining("Failed to create PackagedProgram for application"); + } + + @Test + void testConstructionFailsWithMissingBlob() { + PermanentBlobKey blobKey = new PermanentBlobKey(); + + PackagedProgramApplicationEntry entry = createEntryWithJarBlob(blobKey); + + assertThatThrownBy( + () -> + entry.getApplication( + blobServer, + Collections.emptyList(), + Collections.emptyList())) + .hasMessageContaining("Failed to get user jar file from blob"); + } + + private PackagedProgramApplicationEntry createEntryWithJarBlob(PermanentBlobKey blobKey) { + return new PackagedProgramApplicationEntry( + config, + blobKey, + entryClass, + programArgs, + applicationId, + applicationName, + true, + false, + false, + true); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java index 30480de619e78..9e54491902e9f 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/HighAvailabilityOptions.java @@ -138,6 +138,14 @@ public class HighAvailabilityOptions { "high-availability.zookeeper.path.jobgraphs") .withDescription("ZooKeeper root path (ZNode) for execution plans"); + /** ZooKeeper root path (ZNode) for applications. */ + @Documentation.Section(Documentation.Sections.EXPERT_ZOOKEEPER_HIGH_AVAILABILITY) + public static final ConfigOption HA_ZOOKEEPER_APPLICATIONS_PATH = + key("high-availability.zookeeper.path.applications") + .stringType() + .defaultValue("/applications") + .withDescription("ZooKeeper root path (ZNode) for applications"); + // ------------------------------------------------------------------------ // ZooKeeper Client Settings // ------------------------------------------------------------------------ diff --git a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/HAApplicationRunOnMinioS3StoreITCase.java b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/HAApplicationRunOnMinioS3StoreITCase.java new file mode 100644 index 0000000000000..b80a952fb2ef7 --- /dev/null +++ b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/HAApplicationRunOnMinioS3StoreITCase.java @@ -0,0 +1,150 @@ +/* + * 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.fs.s3.common; + +import org.apache.flink.api.common.ApplicationState; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.testutils.AllCallbackWrapper; +import org.apache.flink.core.testutils.TestContainerExtension; +import org.apache.flink.runtime.highavailability.AbstractHAApplicationRunITCase; +import org.apache.flink.runtime.highavailability.ApplicationResultStoreOptions; +import org.apache.flink.runtime.highavailability.FileSystemApplicationResultStore; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.junit5.MiniClusterExtension; + +import org.apache.flink.shaded.guava33.com.google.common.collect.Iterables; + +import com.amazonaws.services.s3.model.S3ObjectSummary; +import org.apache.commons.lang3.StringUtils; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Order; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.util.List; + +import static org.apache.flink.shaded.guava33.com.google.common.base.Predicates.not; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * {@code ApplicationRunOnMinioS3StoreITCase} covers an application run where the HA data is stored + * in Minio. The implementation verifies whether the {@code JobResult} was written into the + * FileSystem-backed {@code ApplicationResultStore}. + */ +public abstract class HAApplicationRunOnMinioS3StoreITCase extends AbstractHAApplicationRunITCase { + + private static final String CLUSTER_ID = "test-cluster"; + private static final String APPLICATION_RESULT_STORE_FOLDER = "ars"; + + @RegisterExtension + @Order(2) + private static final AllCallbackWrapper> + MINIO_EXTENSION = + new AllCallbackWrapper<>(new TestContainerExtension<>(MinioTestContainer::new)); + + @RegisterExtension + @Order(3) + private static final MiniClusterExtension miniClusterExtension = + new MiniClusterExtension( + () -> { + final Configuration configuration = createConfiguration(); + FileSystem.initialize(configuration, null); + return new MiniClusterResourceConfiguration.Builder() + .setConfiguration(configuration) + .build(); + }); + + private static MinioTestContainer getMinioContainer() { + return MINIO_EXTENSION.getCustomExtension().getTestContainer(); + } + + private static String createS3URIWithSubPath(String... subfolders) { + return getMinioContainer().getS3UriForDefaultBucket() + createSubPath(subfolders); + } + + private static List getObjectsFromApplicationResultStore() { + return getMinioContainer() + .getClient() + .listObjects( + getMinioContainer().getDefaultBucketName(), + createSubPath(CLUSTER_ID, APPLICATION_RESULT_STORE_FOLDER)) + .getObjectSummaries(); + } + + private static String createSubPath(String... subfolders) { + final String pathSeparator = "/"; + return pathSeparator + StringUtils.join(subfolders, pathSeparator); + } + + private static Configuration createConfiguration() { + final Configuration config = new Configuration(); + + getMinioContainer().setS3ConfigOptions(config); + + // ApplicationResultStore configuration + config.set(ApplicationResultStoreOptions.DELETE_ON_COMMIT, Boolean.FALSE); + config.set( + ApplicationResultStoreOptions.STORAGE_PATH, + createS3URIWithSubPath(CLUSTER_ID, APPLICATION_RESULT_STORE_FOLDER)); + + return addHaConfiguration(config, createS3URIWithSubPath(CLUSTER_ID)); + } + + @AfterAll + static void unsetFileSystem() { + FileSystem.initialize(new Configuration(), null); + } + + @Override + protected void runAfterApplicationTermination() throws Exception { + CommonTestUtils.waitUntilCondition( + () -> { + final List objects = getObjectsFromApplicationResultStore(); + return objects.stream() + .map(S3ObjectSummary::getKey) + .anyMatch( + FileSystemApplicationResultStore + ::hasValidApplicationResultStoreEntryExtension) + && objects.stream() + .map(S3ObjectSummary::getKey) + .noneMatch( + FileSystemApplicationResultStore + ::hasValidDirtyApplicationResultStoreEntryExtension); + }, + 2000L); + + final S3ObjectSummary objRef = + Iterables.getOnlyElement(getObjectsFromApplicationResultStore()); + assertThat(objRef.getKey()) + .matches( + FileSystemApplicationResultStore + ::hasValidApplicationResultStoreEntryExtension) + .matches( + not( + FileSystemApplicationResultStore + ::hasValidDirtyApplicationResultStoreEntryExtension)); + + final String objContent = + getMinioContainer() + .getClient() + .getObjectAsString(objRef.getBucketName(), objRef.getKey()); + assertThat(objContent).contains(ApplicationState.FINISHED.name()); + } +} diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HAApplicationRunOnHadoopS3FileSystemITCase.java b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HAApplicationRunOnHadoopS3FileSystemITCase.java new file mode 100644 index 0000000000000..cd4bb089ff207 --- /dev/null +++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HAApplicationRunOnHadoopS3FileSystemITCase.java @@ -0,0 +1,24 @@ +/* + * 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.fs.s3hadoop; + +import org.apache.flink.fs.s3.common.HAApplicationRunOnMinioS3StoreITCase; + +/** Runs the {@link HAApplicationRunOnMinioS3StoreITCase} on the Hadoop S3 file system. */ +class HAApplicationRunOnHadoopS3FileSystemITCase extends HAApplicationRunOnMinioS3StoreITCase {} diff --git a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/HAApplicationRunOnPrestoS3FileSystemITCase.java b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/HAApplicationRunOnPrestoS3FileSystemITCase.java new file mode 100644 index 0000000000000..444c6c6a4d30c --- /dev/null +++ b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/HAApplicationRunOnPrestoS3FileSystemITCase.java @@ -0,0 +1,24 @@ +/* + * 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.fs.s3presto; + +import org.apache.flink.fs.s3.common.HAApplicationRunOnMinioS3StoreITCase; + +/** Runs the {@link HAApplicationRunOnMinioS3StoreITCase} on the Presto S3 file system. */ +class HAApplicationRunOnPrestoS3FileSystemITCase extends HAApplicationRunOnMinioS3StoreITCase {} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesApplicationStoreUtil.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesApplicationStoreUtil.java new file mode 100644 index 0000000000000..dae6ff5a8c86e --- /dev/null +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesApplicationStoreUtil.java @@ -0,0 +1,54 @@ +/* + * 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.ApplicationID; +import org.apache.flink.kubernetes.utils.Constants; +import org.apache.flink.runtime.jobmanager.ApplicationStoreUtil; + +import static org.apache.flink.kubernetes.utils.Constants.APPLICATION_STORE_KEY_PREFIX; + +/** Singleton {@link ApplicationStoreUtil} implementation for Kubernetes. */ +public enum KubernetesApplicationStoreUtil implements ApplicationStoreUtil { + INSTANCE; + + /** + * Convert a key in ConfigMap to {@link ApplicationID}. The key is stored with prefix {@link + * Constants#APPLICATION_STORE_KEY_PREFIX}. + * + * @param key application key in ConfigMap. + * @return the parsed {@link ApplicationID}. + */ + @Override + public ApplicationID nameToApplicationId(String key) { + return ApplicationID.fromHexString(key.substring(APPLICATION_STORE_KEY_PREFIX.length())); + } + + /** + * Convert a {@link ApplicationID} to config map key. We will add prefix {@link + * Constants#APPLICATION_STORE_KEY_PREFIX}. + * + * @param applicationId application id + * @return a key to store application in the ConfigMap + */ + @Override + public String applicationIdToName(ApplicationID applicationId) { + return APPLICATION_STORE_KEY_PREFIX + applicationId; + } +} diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionHaServices.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionHaServices.java index e29f72058b3b8..44709ed5383e7 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionHaServices.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionHaServices.java @@ -28,7 +28,9 @@ 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.FileSystemApplicationResultStore; import org.apache.flink.runtime.highavailability.FileSystemJobResultStore; +import org.apache.flink.runtime.jobmanager.ApplicationStore; import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; import org.apache.flink.runtime.leaderelection.LeaderElectionDriverFactory; import org.apache.flink.runtime.leaderretrieval.DefaultLeaderRetrievalService; @@ -104,7 +106,8 @@ private KubernetesLeaderElectionHaServices( configuration), ioExecutor, blobStoreService, - FileSystemJobResultStore.fromConfiguration(configuration, ioExecutor)); + FileSystemJobResultStore.fromConfiguration(configuration, ioExecutor), + FileSystemApplicationResultStore.fromConfiguration(configuration, ioExecutor)); this.kubeClient = checkNotNull(kubeClient); this.clusterId = checkNotNull(clusterId); @@ -156,6 +159,12 @@ protected ExecutionPlanStore createExecutionPlanStore() throws Exception { configuration, kubeClient, getClusterConfigMap(), lockIdentity); } + @Override + protected ApplicationStore createApplicationStore() throws Exception { + return KubernetesUtils.createApplicationStore( + configuration, kubeClient, getClusterConfigMap(), lockIdentity); + } + private String getClusterConfigMap() { return getClusterConfigMap(clusterId); } 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 21d36b66d11b5..6359364248c6b 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 @@ -110,6 +110,8 @@ public class Constants { public static final String LEADER_SESSION_ID_KEY = "sessionId"; public static final String EXECUTION_PLAN_STORE_KEY_PREFIX = "executionPlan-"; public static final String SUBMITTED_EXECUTION_PLAN_FILE_PREFIX = "submittedExecutionPlan"; + public static final String APPLICATION_STORE_KEY_PREFIX = "application-"; + public static final String SUBMITTED_APPLICATION_FILE_PREFIX = "submittedApplication"; 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 f097354fdc387..7786c0add5648 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 @@ -25,6 +25,7 @@ import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.core.execution.RecoveryClaimMode; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; +import org.apache.flink.kubernetes.highavailability.KubernetesApplicationStoreUtil; import org.apache.flink.kubernetes.highavailability.KubernetesCheckpointStoreUtil; import org.apache.flink.kubernetes.highavailability.KubernetesExecutionPlanStoreUtil; import org.apache.flink.kubernetes.highavailability.KubernetesStateHandleStore; @@ -37,6 +38,9 @@ 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.jobmanager.ApplicationStore; +import org.apache.flink.runtime.jobmanager.ApplicationStoreEntry; +import org.apache.flink.runtime.jobmanager.DefaultApplicationStore; import org.apache.flink.runtime.jobmanager.DefaultExecutionPlanStore; import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; import org.apache.flink.runtime.jobmanager.NoOpExecutionPlanStoreWatcher; @@ -83,6 +87,7 @@ import java.util.concurrent.Executor; import java.util.stream.Collectors; +import static org.apache.flink.kubernetes.utils.Constants.APPLICATION_STORE_KEY_PREFIX; import static org.apache.flink.kubernetes.utils.Constants.CHECKPOINT_ID_KEY_PREFIX; import static org.apache.flink.kubernetes.utils.Constants.COMPLETED_CHECKPOINT_FILE_SUFFIX; import static org.apache.flink.kubernetes.utils.Constants.DNS_POLICY_DEFAULT; @@ -90,6 +95,7 @@ import static org.apache.flink.kubernetes.utils.Constants.EXECUTION_PLAN_STORE_KEY_PREFIX; import static org.apache.flink.kubernetes.utils.Constants.LEADER_ADDRESS_KEY; import static org.apache.flink.kubernetes.utils.Constants.LEADER_SESSION_ID_KEY; +import static org.apache.flink.kubernetes.utils.Constants.SUBMITTED_APPLICATION_FILE_PREFIX; import static org.apache.flink.kubernetes.utils.Constants.SUBMITTED_EXECUTION_PLAN_FILE_PREFIX; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -234,6 +240,62 @@ public static LeaderInformation getLeaderInformationFromConfigMap( return LeaderInformation.known(sessionID, leaderAddress); } + /** + * Create a {@link DefaultApplicationStore}. + * + * @param configuration configuration to build a RetrievableStateStorageHelper + * @param flinkKubeClient flink kubernetes client + * @param configMapName ConfigMap name + * @param lockIdentity lock identity to check the leadership + * @return a {@link DefaultExecutionPlanStore} + * @throws Exception when create the storage helper + */ + public static ApplicationStore createApplicationStore( + Configuration configuration, + FlinkKubeClient flinkKubeClient, + String configMapName, + String lockIdentity) + throws Exception { + + final KubernetesStateHandleStore stateHandleStore = + createApplicationStateHandleStore( + configuration, flinkKubeClient, configMapName, lockIdentity); + return new DefaultApplicationStore<>( + stateHandleStore, KubernetesApplicationStoreUtil.INSTANCE); + } + + /** + * Create a {@link KubernetesStateHandleStore} which storing {@link ApplicationStoreEntry}. + * + * @param configuration configuration to build a RetrievableStateStorageHelper + * @param flinkKubeClient flink kubernetes client + * @param configMapName ConfigMap name + * @param lockIdentity lock identity to check the leadership + * @return a {@link KubernetesStateHandleStore} which storing {@link ApplicationStoreEntry}. + * @throws Exception when create the storage helper + */ + public static KubernetesStateHandleStore + createApplicationStateHandleStore( + Configuration configuration, + FlinkKubeClient flinkKubeClient, + String configMapName, + String lockIdentity) + throws Exception { + + final RetrievableStateStorageHelper stateStorage = + new FileSystemStateStorageHelper<>( + HighAvailabilityServicesUtils.getClusterHighAvailableStoragePath( + configuration), + SUBMITTED_APPLICATION_FILE_PREFIX); + + return new KubernetesStateHandleStore<>( + flinkKubeClient, + configMapName, + stateStorage, + k -> k.startsWith(APPLICATION_STORE_KEY_PREFIX), + lockIdentity); + } + /** * Create a {@link DefaultExecutionPlanStore} with {@link NoOpExecutionPlanStoreWatcher}. * diff --git a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunApplicationHandler.java b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunApplicationHandler.java index 58b3b7b8e8099..33f684174f53b 100644 --- a/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunApplicationHandler.java +++ b/flink-runtime-web/src/main/java/org/apache/flink/runtime/webmonitor/handlers/JarRunApplicationHandler.java @@ -25,6 +25,7 @@ import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.DeploymentOptions; +import org.apache.flink.configuration.PipelineOptionsInternal; import org.apache.flink.configuration.StateRecoveryOptions; import org.apache.flink.core.execution.RecoveryClaimMode; import org.apache.flink.runtime.blob.BlobClient; @@ -49,6 +50,7 @@ import java.nio.file.Path; import java.time.Duration; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; @@ -105,20 +107,34 @@ public CompletableFuture handleRequest( final PackagedProgram program = context.toPackagedProgram(effectiveConfiguration); ApplicationID applicationId = context.getApplicationId().orElse(ApplicationID.generate()); - PackagedProgramApplication application = - new PackagedProgramApplication( - applicationId, program, effectiveConfiguration, false, true, false, false); - // TODO upload user jar to blob in HA mode once application resource cleanup is supported final boolean isHaEnabled = HighAvailabilityMode.isHighAvailabilityModeActivated(configuration); - CompletableFuture jarUploadFuture = - CompletableFuture.completedFuture(null); + final CompletableFuture jarUploadFuture; + if (isHaEnabled) { + // In HA mode, a fixed job id is required to ensure consistency across failovers. + // The job id is derived from the application id if not configured. + maybeFixJobId(effectiveConfiguration, applicationId); + + // upload user jar file to blob server for HA recovery + jarUploadFuture = uploadJarFile(gateway, context, applicationId); + } else { + jarUploadFuture = CompletableFuture.completedFuture(null); + } return jarUploadFuture .thenCompose( blobKey -> { - // TODO record blob key in the application for HA recovery + PackagedProgramApplication application = + new PackagedProgramApplication( + applicationId, + program, + effectiveConfiguration, + false, + true, + false, + false, + blobKey); return gateway.submitApplication(application, timeout); }) .handle( @@ -134,6 +150,16 @@ public CompletableFuture handleRequest( }); } + private void maybeFixJobId( + final Configuration configuration, final ApplicationID applicationId) { + final Optional configuredJobId = + configuration.getOptional(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID); + if (configuredJobId.isEmpty()) { + configuration.set( + PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID, applicationId.toHexString()); + } + } + private CompletableFuture uploadJarFile( final DispatcherGateway gateway, final JarHandlerContext context, diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java index 22ad693e68652..4c7cd990d2349 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/history/HistoryServerTest.java @@ -137,7 +137,9 @@ void testHistoryServerIntegration(final boolean versionLessThan14) throws Except final int numJobs = 2; final int numLegacyJobs = 1; - CountDownLatch numExpectedArchivedJobs = new CountDownLatch(numJobs + numLegacyJobs); + CountDownLatch numExpectedLegacyArchives = new CountDownLatch(numLegacyJobs); + // one for application itself, one for the job + CountDownLatch numExpectedArchives = new CountDownLatch(numJobs * 2); Configuration historyServerConfig = createTestConfiguration(false); @@ -147,11 +149,14 @@ void testHistoryServerIntegration(final boolean versionLessThan14) throws Except (event) -> { if (event.getType() == HistoryServerArchiveFetcher.ArchiveEventType.CREATED) { - numExpectedArchivedJobs.countDown(); + numExpectedLegacyArchives.countDown(); } }, (event) -> { - throw new RuntimeException("Should not call"); + if (event.getType() + == HistoryServerArchiveFetcher.ArchiveEventType.CREATED) { + numExpectedArchives.countDown(); + } }); try { @@ -161,13 +166,16 @@ void testHistoryServerIntegration(final boolean versionLessThan14) throws Except assertThat(getJobsOverview(baseUrl).getJobs()).isEmpty(); for (int x = 0; x < numJobs; x++) { - mockJobArchive(createExecutionGraphInfo(), null); + // the job submitted in MiniCluster will be wrapped into an application + runJob(); } createLegacyArchive(jmDirectory.toPath(), versionLessThan14); - waitForArchivesCreation(numJobs + numLegacyJobs); + waitForArchivesCreation(numJobs); - assertThat(numExpectedArchivedJobs.await(10L, TimeUnit.SECONDS)).isTrue(); + assertThat(numExpectedLegacyArchives.await(10L, TimeUnit.SECONDS)).isTrue(); + assertThat(numExpectedArchives.await(10L, TimeUnit.SECONDS)).isTrue(); assertThat(getJobsOverview(baseUrl).getJobs()).hasSize(numJobs + numLegacyJobs); + assertThat(getApplicationsOverview(baseUrl).getApplications()).hasSize(numJobs); // checks whether the dashboard configuration contains all expected fields getDashboardConfiguration(baseUrl); @@ -340,7 +348,6 @@ private void runArchiveExpirationTest(boolean cleanupExpiredJobs) throws Excepti for (int x = 0; x < numJobs; x++) { mockJobArchive(createExecutionGraphInfo(), null); } - waitForArchivesCreation(numJobs); CountDownLatch numExpectedArchivedJobs = new CountDownLatch(numJobs); CountDownLatch firstArchiveExpiredLatch = new CountDownLatch(numExpiredJobs); @@ -447,13 +454,35 @@ private void assertFilesCleanedUp(boolean filesShouldBeDeleted) throws IOExcepti } } - private void waitForArchivesCreation(int numJobs) throws InterruptedException { - // the job is archived asynchronously after env.execute() returns - File[] archives = jmDirectory.listFiles(); - while (archives == null || archives.length != numJobs) { + private void waitForArchivesCreation(int numApplications) throws InterruptedException { + // the job is archived asynchronously after env.execute() returns and the application is + // archived after its job completes, so we wait for the application to be archived here + int count = 0; + while (count < numApplications) { Thread.sleep(50); - archives = jmDirectory.listFiles(); + count = getApplicationArchiveCount(); + } + } + + private int getApplicationArchiveCount() { + int count = 0; + File[] clusterDirs = jmDirectory.listFiles(); + if (clusterDirs == null) { + return 0; + } + + for (File clusterDir : clusterDirs) { + if (clusterDir.isDirectory()) { + File applicationsDir = new File(clusterDir, ArchivePathUtils.APPLICATIONS_DIR); + if (applicationsDir.exists() && applicationsDir.isDirectory()) { + File[] applicationDirs = applicationsDir.listFiles(); + if (applicationDirs != null) { + count += applicationDirs.length; + } + } + } } + return count; } private Configuration createTestConfiguration(boolean cleanupExpiredJobs) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/application/AbstractApplication.java b/flink-runtime/src/main/java/org/apache/flink/runtime/application/AbstractApplication.java index 58134466583b6..e569d2002e533 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/application/AbstractApplication.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/application/AbstractApplication.java @@ -23,6 +23,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.jobmanager.ApplicationStore; +import org.apache.flink.runtime.jobmanager.ApplicationStoreEntry; import org.apache.flink.runtime.messages.Acknowledge; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.util.concurrent.ScheduledExecutor; @@ -40,6 +42,7 @@ import java.util.HashSet; 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.Executor; @@ -169,6 +172,17 @@ private List getStatusListeners() { return statusListeners; } + /** + * Converts the application to an {@link ApplicationStoreEntry} that can be persisted by the + * {@link ApplicationStore}. + * + * @return the application store entry or {@link Optional#empty()} if the application should not + * be persisted. + */ + public Optional getApplicationStoreEntry() { + return Optional.empty(); + } + // ------------------------------------------------------------------------ // State Transitions // ------------------------------------------------------------------------ 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 007140573a23b..9523fa2575583 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 @@ -1314,6 +1314,29 @@ public void retainJobs(Collection jobsToRetain, Executor ioExecutor) thro } } + public void retainApplications( + Collection applicationsToRetain, Executor ioExecutor) + throws IOException { + if (storageDir.deref().exists()) { + final Set applicationsToRemove = + BlobUtils.listExistingApplications(storageDir.deref().toPath()); + + applicationsToRemove.removeAll(applicationsToRetain); + + final Collection> cleanupResultFutures = + new ArrayList<>(applicationsToRemove.size()); + for (ApplicationID applicationToRemove : applicationsToRemove) { + cleanupResultFutures.add(globalCleanupAsync(applicationToRemove, ioExecutor)); + } + + try { + FutureUtils.completeAll(cleanupResultFutures).get(); + } catch (InterruptedException | ExecutionException e) { + ExceptionUtils.rethrowIOException(e); + } + } + } + @Override public PermanentBlobService getPermanentBlobService() { return this; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java index ee490f00e6544..47fb9bd06126a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobUtils.java @@ -762,6 +762,14 @@ static Set listExistingJobs(java.nio.file.Path directory) throws IOExcept .collect(Collectors.toSet()); } + static Set listExistingApplications(java.nio.file.Path directory) + throws IOException { + return listBlobsInDirectory(directory).stream() + .map(Blob::getApplicationId) + .filter(Objects::nonNull) + .collect(Collectors.toSet()); + } + abstract static class Blob { private final T blobKey; private final java.nio.file.Path path; 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 d0c0f2242456a..355b264262e38 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 @@ -51,7 +51,10 @@ 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.ApplicationResourceCleaner; +import org.apache.flink.runtime.dispatcher.cleanup.ApplicationResourceCleanerFactory; import org.apache.flink.runtime.dispatcher.cleanup.CleanupRunnerFactory; +import org.apache.flink.runtime.dispatcher.cleanup.DispatcherApplicationResourceCleanerFactory; import org.apache.flink.runtime.dispatcher.cleanup.DispatcherResourceCleanerFactory; import org.apache.flink.runtime.dispatcher.cleanup.ResourceCleaner; import org.apache.flink.runtime.dispatcher.cleanup.ResourceCleanerFactory; @@ -62,6 +65,9 @@ import org.apache.flink.runtime.executiongraph.ErrorInfo; import org.apache.flink.runtime.executiongraph.JobStatusListener; import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.ApplicationResult; +import org.apache.flink.runtime.highavailability.ApplicationResultEntry; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.JobResultEntry; import org.apache.flink.runtime.highavailability.JobResultStore; @@ -70,6 +76,8 @@ import org.apache.flink.runtime.jobgraph.JobResourceRequirements; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.jobmanager.ApplicationStoreEntry; +import org.apache.flink.runtime.jobmanager.ApplicationWriter; import org.apache.flink.runtime.jobmanager.ExecutionPlanWriter; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerRunnerResult; @@ -136,7 +144,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -173,6 +180,8 @@ public abstract class Dispatcher extends FencedRpcEndpoint private final ExecutionPlanWriter executionPlanWriter; private final JobResultStore jobResultStore; + private final ApplicationWriter applicationWriter; + private final ApplicationResultStore applicationResultStore; private final HighAvailabilityServices highAvailabilityServices; private final GatewayRetriever resourceManagerGatewayRetriever; @@ -185,6 +194,14 @@ public abstract class Dispatcher extends FencedRpcEndpoint private final OnMainThreadJobManagerRunnerRegistry jobManagerRunnerRegistry; + /** + * Map of applications that were suspended in a previous execution attempt, which must be + * recovered in the current execution attempt. + */ + private final Map suspendedApplications = new HashMap<>(); + + private final Collection recoveredDirtyApplicationResults; + /** * Map of jobs that were suspended in a previous application execution attempt, which may be * recovered or deprecated in the current execution attempt. @@ -227,6 +244,8 @@ public abstract class Dispatcher extends FencedRpcEndpoint private final ResourceCleaner localResourceCleaner; private final ResourceCleaner globalResourceCleaner; + private final ApplicationResourceCleaner applicationResourceCleaner; + private final Duration webTimeout; private final Map jobClientExpiredTimestamp = new HashMap<>(); @@ -243,7 +262,13 @@ public abstract class Dispatcher extends FencedRpcEndpoint private final Map applications = new HashMap<>(); /** ExecutionGraphInfo for the terminated job whose application is not terminated yet. */ - private final Map> partialExecutionGraphInfoStore = + private final Map partialExecutionGraphInfoStore = new HashMap<>(); + + /** Futures to control the termination workflow for applications and jobs. */ + private final Map> jobCreateDirtyResultFutures = new HashMap<>(); + + private final Map> jobMarkResultCleanFutures = new HashMap<>(); + private final Map> applicationCreateDirtyResultFutures = new HashMap<>(); private final Map> applicationTerminationFutures = @@ -262,6 +287,8 @@ public Dispatcher( DispatcherId fencingToken, Collection recoveredJobs, Collection recoveredDirtyJobs, + Collection recoveredApplications, + Collection recoveredDirtyApplicationResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, DispatcherServices dispatcherServices) throws Exception { @@ -270,6 +297,8 @@ public Dispatcher( fencingToken, recoveredJobs, recoveredDirtyJobs, + recoveredApplications, + recoveredDirtyApplicationResults, dispatcherBootstrapFactory, dispatcherServices, new DefaultJobManagerRunnerRegistry(INITIAL_JOB_MANAGER_RUNNER_REGISTRY_CAPACITY)); @@ -280,6 +309,8 @@ private Dispatcher( DispatcherId fencingToken, Collection recoveredJobs, Collection recoveredDirtyJobs, + Collection recoveredApplications, + Collection recoveredDirtyApplicationResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, DispatcherServices dispatcherServices, JobManagerRunnerRegistry jobManagerRunnerRegistry) @@ -289,10 +320,13 @@ private Dispatcher( fencingToken, recoveredJobs, recoveredDirtyJobs, + recoveredApplications, + recoveredDirtyApplicationResults, dispatcherBootstrapFactory, dispatcherServices, jobManagerRunnerRegistry, - new DispatcherResourceCleanerFactory(jobManagerRunnerRegistry, dispatcherServices)); + new DispatcherResourceCleanerFactory(jobManagerRunnerRegistry, dispatcherServices), + new DispatcherApplicationResourceCleanerFactory(dispatcherServices)); } @VisibleForTesting @@ -301,13 +335,18 @@ protected Dispatcher( DispatcherId fencingToken, Collection recoveredJobs, Collection recoveredDirtyJobs, + Collection recoveredApplications, + Collection recoveredDirtyApplicationResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, DispatcherServices dispatcherServices, JobManagerRunnerRegistry jobManagerRunnerRegistry, - ResourceCleanerFactory resourceCleanerFactory) + ResourceCleanerFactory resourceCleanerFactory, + ApplicationResourceCleanerFactory applicationResourceCleanerFactory) throws Exception { super(rpcService, RpcServiceUtils.createRandomName(DISPATCHER_NAME), fencingToken); assertRecoveredJobsAndDirtyJobResults(recoveredJobs, recoveredDirtyJobs); + assertRecoveredApplicationsAndDirtyApplicationResults( + recoveredApplications, recoveredDirtyApplicationResults); this.configuration = dispatcherServices.getConfiguration(); this.highAvailabilityServices = dispatcherServices.getHighAvailabilityServices(); @@ -319,6 +358,8 @@ protected Dispatcher( this.failureEnrichers = dispatcherServices.getFailureEnrichers(); this.executionPlanWriter = dispatcherServices.getExecutionPlanWriter(); this.jobResultStore = dispatcherServices.getJobResultStore(); + this.applicationWriter = dispatcherServices.getApplicationWriter(); + this.applicationResultStore = dispatcherServices.getApplicationResultStore(); this.jobManagerMetricGroup = dispatcherServices.getJobManagerMetricGroup(); this.metricServiceQueryAddress = dispatcherServices.getMetricQueryServiceAddress(); this.ioExecutor = dispatcherServices.getIoExecutor(); @@ -347,6 +388,12 @@ protected Dispatcher( this.dispatcherBootstrapFactory = checkNotNull(dispatcherBootstrapFactory); + this.recoveredDirtyApplicationResults = new ArrayList<>(recoveredDirtyApplicationResults); + + for (AbstractApplication application : recoveredApplications) { + this.suspendedApplications.put(application.getApplicationId(), application); + } + for (ExecutionPlan executionPlan : recoveredJobs) { final JobID jobId = executionPlan.getJobID(); final ApplicationID applicationId = @@ -379,6 +426,12 @@ protected Dispatcher( recoveredJobs.stream().map(ExecutionPlan::getJobID).collect(Collectors.toSet()), dispatcherServices.getIoExecutor()); + this.blobServer.retainApplications( + recoveredApplications.stream() + .map(AbstractApplication::getApplicationId) + .collect(Collectors.toSet()), + dispatcherServices.getIoExecutor()); + this.dispatcherCachedOperationsHandler = new DispatcherCachedOperationsHandler( dispatcherServices.getOperationCaches(), @@ -390,6 +443,9 @@ protected Dispatcher( resourceCleanerFactory.createLocalResourceCleaner(this.getMainThreadExecutor()); this.globalResourceCleaner = resourceCleanerFactory.createGlobalResourceCleaner(this.getMainThreadExecutor()); + this.applicationResourceCleaner = + applicationResourceCleanerFactory.createApplicationResourceCleaner( + this.getMainThreadExecutor()); this.webTimeout = configuration.get(WebOptions.TIMEOUT); @@ -428,24 +484,53 @@ public void onStart() throws Exception { this::onFatalError); if (dispatcherBootstrap instanceof ApplicationBootstrap) { - // defer starting recovered jobs, as they might be skipped based on user logic - internalSubmitApplication(((ApplicationBootstrap) dispatcherBootstrap).getApplication()) - .get(); + // Application Mode + checkState(suspendedApplications.isEmpty()); + checkState(recoveredDirtyApplicationResults.size() <= 1); + + AbstractApplication application = + ((ApplicationBootstrap) dispatcherBootstrap).getApplication(); + if (!recoveredDirtyApplicationResults.isEmpty()) { + // the application is already terminated + ApplicationResult applicationResult = + recoveredDirtyApplicationResults.iterator().next(); + checkState(application.getApplicationId() == applicationResult.getApplicationId()); + + startApplicationCleanup(); + } else { + // defer starting recovered jobs, as they might be skipped based on user logic + internalSubmitApplication(application).get(); + } } else { - // Jobs cannot be associated with applications because applications do not yet exist - // during recovery. This is a temporary workaround and will be removed once application - // recovery is supported. - recoveredDirtyJobResultsByApplicationId.values().stream() - .flatMap(Collection::stream) - .forEach(jobResult -> runJobWithCleanupRunner(jobResult, false)); - - // start recovered jobs by wrapping them into a SingleJobApplication - for (ExecutionPlan recoveredJob : suspendedJobs.values()) { - runRecoveredJob(recoveredJob, true); + // Session Mode + startApplicationCleanup(); + + // start suspended applications + for (AbstractApplication suspendedApplication : suspendedApplications.values()) { + // defer starting recovered jobs, as they might be skipped based on user logic + internalSubmitApplication(suspendedApplication).get(); + } + + // start suspended jobs that do not belong to any application (previously submitted in a + // SingleJobApplication) by wrapping them into a SingleJobApplication + Iterator> jobIterator = + suspendedJobs.entrySet().iterator(); + while (jobIterator.hasNext()) { + Map.Entry entry = jobIterator.next(); + ExecutionPlan recoveredJob = entry.getValue(); + ApplicationID applicationId = recoveredJob.getApplicationId().orElse(null); + + if (!suspendedApplications.containsKey(applicationId)) { + runRecoveredJob(recoveredJob, true); + jobIterator.remove(); + suspendedJobIdsByApplicationId.remove(applicationId); + } } - suspendedJobs.clear(); - suspendedJobIdsByApplicationId.clear(); } + + checkState(recoveredDirtyJobResultsByApplicationId.isEmpty()); + recoveredDirtyApplicationResults.clear(); + suspendedApplications.clear(); } private void startDispatcherServices() throws Exception { @@ -480,6 +565,26 @@ private static void assertRecoveredJobsAndDirtyJobResults( "There should be no overlap between the recovered ExecutionPlans and the passed dirty JobResults based on their job ID."); } + private static void assertRecoveredApplicationsAndDirtyApplicationResults( + Collection recoveredApplications, + Collection recoveredDirtyApplicationResults) { + final Set applicationIdsOfFinishedApplications = + recoveredDirtyApplicationResults.stream() + .map(ApplicationResult::getApplicationId) + .collect(Collectors.toSet()); + + final boolean noRecoveredApplicationHasDirtyApplicationResult = + recoveredApplications.stream() + .noneMatch( + recoveredApplication -> + applicationIdsOfFinishedApplications.contains( + recoveredApplication.getApplicationId())); + + Preconditions.checkArgument( + noRecoveredApplicationHasDirtyApplicationResult, + "There should be no overlap between the recovered Applications and the passed dirty ApplicationResults based on their application ID."); + } + /** * Runs a recovered job in HA mode. * @@ -565,6 +670,51 @@ private void runJobWithCleanupRunner( } } + private void startApplicationCleanup() { + for (ApplicationResult applicationResult : recoveredDirtyApplicationResults) { + ApplicationID applicationId = applicationResult.getApplicationId(); + ApplicationState applicationState = applicationResult.getApplicationState(); + + Map jobs = new HashMap<>(); + Collection dirtyJobResults = + recoveredDirtyJobResultsByApplicationId.remove(applicationId); + if (dirtyJobResults != null) { + for (JobResult jobResult : dirtyJobResults) { + JobID jobId = jobResult.getJobId(); + ExecutionGraphInfo executionGraphInfo = + new ExecutionGraphInfo( + ArchivedExecutionGraph.createFrom(jobResult, -1)); + jobs.put(jobId, executionGraphInfo); + + runJobWithCleanupRunner(jobResult, false); + } + } + + long[] stateTimestamps = new long[ApplicationState.values().length]; + stateTimestamps[ApplicationState.CREATED.ordinal()] = applicationResult.getStartTime(); + stateTimestamps[applicationState.ordinal()] = applicationResult.getEndTime(); + + ArchivedApplication sparseArchivedApplication = + new ArchivedApplication( + applicationId, + applicationResult.getApplicationName(), + applicationState, + stateTimestamps, + jobs, + Collections.emptyList()); + + writeToArchivedApplicationStore(sparseArchivedApplication); + + // the dirty result already exists + // create a completed future to make sure the jobs can be marked clean + applicationCreateDirtyResultFutures.put( + applicationId, FutureUtils.completedVoidFuture()); + applicationTerminationFutures.put(applicationId, new CompletableFuture<>()); + + removeApplication(applicationId, jobs.keySet()); + } + } + private void handleStartDispatcherServicesException(Exception e) throws Exception { try { stopDispatcherServices(); @@ -726,6 +876,21 @@ public CompletableFuture submitApplication( new DuplicateApplicationSubmissionException(applicationId)); } + Optional optionalApplicationStoreEntry = + application.getApplicationStoreEntry(); + if (optionalApplicationStoreEntry.isPresent()) { + try { + applicationWriter.putApplication(optionalApplicationStoreEntry.get()); + } catch (Exception e) { + String msg = + String.format( + "Could not persist application %s to the ApplicationStore.", + applicationId); + log.warn(msg); + throw new CompletionException(new RuntimeException(msg, e)); + } + } + return internalSubmitApplication(application); } @@ -737,6 +902,7 @@ private CompletableFuture internalSubmitApplication( applications.put(applicationId, application); application.registerStatusListener(this); + applicationCreateDirtyResultFutures.put(applicationId, new CompletableFuture<>()); applicationTerminationFutures.put(applicationId, new CompletableFuture<>()); // cleanup dirty job results by creating a JobManagerRunner during application submission to @@ -768,94 +934,24 @@ public void notifyApplicationStatusChange( "The application (" + applicationId + ") has already terminated."); AbstractApplication application = applications.get(applicationId); - Set remainingSuspendedJobIds = - suspendedJobIdsByApplicationId.remove(applicationId); - if (remainingSuspendedJobIds != null) { - for (JobID jobId : remainingSuspendedJobIds) { - final ExecutionPlan executionPlan = suspendedJobs.remove(jobId); - checkNotNull(executionPlan); - - final JobResult jobResult = - new JobResult.Builder() - .jobId(jobId) - .jobStatus(JobStatus.FAILED) - .netRuntime(0) - .serializedThrowable( - new SerializedThrowable( - new FlinkException( - "Job recovery is not needed."))) - .jobName(executionPlan.getName()) - .applicationId(applicationId) - .build(); - runJobWithCleanupRunner(jobResult); - } - } - log.info( - "Archiving application ({}) with terminal state {}.", applicationId, newStatus); - long[] stateTimestamps = new long[ApplicationState.values().length]; - for (ApplicationState applicationState : ApplicationState.values()) { - final int ordinal = applicationState.ordinal(); - stateTimestamps[ordinal] = application.getStatusTimestamp(applicationState); - } + // Step 1: cleanup remaining suspended jobs for the application + cleanupRemainingSuspendedJobs(applicationId); - List> jobFutures = - application.getJobs().stream() - .map( - jobId -> { - checkState( - partialExecutionGraphInfoStore.containsKey(jobId), - "ExecutionGraphInfo for job %s does not exist.", - jobId); - return partialExecutionGraphInfoStore.get(jobId); - }) - .collect(Collectors.toList()); - - // wait for all jobs to be stored, then archive the application - CompletableFuture applicationArchivingFuture = - FutureUtils.combineAll(jobFutures) + // Step 2: finalize the archived application after all jobs are marked as terminated + CompletableFuture applicationArchivingFuture = + FutureUtils.waitForAll( + application.getJobs().stream() + .map(jobCreateDirtyResultFutures::get) + .collect(Collectors.toList())) .thenComposeAsync( - combinedJobs -> { - Map jobs = new HashMap<>(); - for (ExecutionGraphInfo executionGraphInfo : combinedJobs) { - jobs.put( - executionGraphInfo.getJobId(), - executionGraphInfo); - partialExecutionGraphInfoStore.remove( - executionGraphInfo.getJobId()); - } - - // record job exception for SingleJobApplication - if (application instanceof SingleJobApplication) { - jobs.values() - .forEach( - executionGraphInfo -> { - ErrorInfo errorInfo = - executionGraphInfo - .getArchivedExecutionGraph() - .getFailureInfo(); - if (errorInfo != null) { - application - .addExceptionHistoryEntry( - errorInfo - .getException(), - executionGraphInfo - .getJobId()); - } - }); - } - - ArchivedApplication archivedApplication = - new ArchivedApplication( - application.getApplicationId(), - application.getName(), - application.getApplicationStatus(), - stateTimestamps, - jobs, - application.getExceptionHistory()); - + ignored -> { + final ArchivedApplication archivedApplication = + finalizeArchivedApplication(application, newStatus); applications.remove(applicationId); + writeToArchivedApplicationStore(archivedApplication); + return historyServerArchivist .archiveApplication(archivedApplication) .exceptionally( @@ -865,16 +961,95 @@ public void notifyApplicationStatusChange( applicationId, throwable); return null; - }); + }) + .thenApply(ack -> archivedApplication); }, getMainThreadExecutor()); - applicationArchivingFuture.thenRunAsync( - () -> applicationTerminationFutures.get(applicationId).complete(null), - getMainThreadExecutor()); + // Step 3: create a dirty result for the application + CompletableFuture applicationDirtyResultFuture = + applicationArchivingFuture.thenCompose( + this::registerGloballyTerminatedApplicationInApplicationResultStore); + + // Step 4: perform application cleanup and mark result clean after all cleanup + // (including job cleanup) is done + applicationDirtyResultFuture.thenCompose( + ignored -> removeApplication(applicationId, application.getJobs())); } } + private void cleanupRemainingSuspendedJobs(ApplicationID applicationId) { + Set remainingSuspendedJobIds = suspendedJobIdsByApplicationId.remove(applicationId); + if (remainingSuspendedJobIds != null) { + for (JobID jobId : remainingSuspendedJobIds) { + final ExecutionPlan executionPlan = suspendedJobs.remove(jobId); + checkNotNull(executionPlan); + + final JobResult jobResult = + new JobResult.Builder() + .jobId(jobId) + .jobStatus(JobStatus.FAILED) + .netRuntime(0) + .serializedThrowable( + new SerializedThrowable( + new FlinkException("Job recovery is not needed."))) + .jobName(executionPlan.getName()) + .applicationId(applicationId) + .build(); + runJobWithCleanupRunner(jobResult); + } + } + } + + private ArchivedApplication finalizeArchivedApplication( + AbstractApplication application, ApplicationState status) { + final ApplicationID applicationId = application.getApplicationId(); + + log.info("Archiving application ({}) with terminal state {}.", applicationId, status); + + long[] stateTimestamps = new long[ApplicationState.values().length]; + for (ApplicationState applicationState : ApplicationState.values()) { + final int ordinal = applicationState.ordinal(); + stateTimestamps[ordinal] = application.getStatusTimestamp(applicationState); + } + + final Map jobs = new HashMap<>(application.getJobs().size()); + + for (JobID jobId : application.getJobs()) { + ExecutionGraphInfo executionGraphInfo = + checkNotNull( + partialExecutionGraphInfoStore.remove(jobId), + "ExecutionGraphInfo for job %s does not exist.", + jobId); + jobs.put(jobId, executionGraphInfo); + } + + // record job exception for SingleJobApplication + if (application instanceof SingleJobApplication) { + jobs.values() + .forEach( + executionGraphInfo -> { + ErrorInfo errorInfo = + executionGraphInfo + .getArchivedExecutionGraph() + .getFailureInfo(); + if (errorInfo != null) { + application.addExceptionHistoryEntry( + errorInfo.getException(), + executionGraphInfo.getJobId()); + } + }); + } + + return new ArchivedApplication( + applicationId, + application.getName(), + status, + stateTimestamps, + jobs, + application.getExceptionHistory()); + } + private void writeToArchivedApplicationStore(ArchivedApplication archivedApplication) { try { archivedApplicationStore.put(archivedApplication); @@ -887,6 +1062,83 @@ private void writeToArchivedApplicationStore(ArchivedApplication archivedApplica } } + private CompletableFuture registerGloballyTerminatedApplicationInApplicationResultStore( + ArchivedApplication application) { + final ApplicationID applicationId = application.getApplicationId(); + + return applicationResultStore + .hasCleanApplicationResultEntryAsync(applicationId) + .thenCompose( + hasCleanResult -> { + if (hasCleanResult) { + log.warn( + "Application {} is already marked as clean but clean up was triggered again.", + applicationId); + return FutureUtils.completedVoidFuture(); + } + + return applicationResultStore + .hasDirtyApplicationResultEntryAsync(applicationId) + .thenCompose( + hasDirtyResult -> { + if (hasDirtyResult) { + return FutureUtils.completedVoidFuture(); + } + + return applicationResultStore + .createDirtyResultAsync( + new ApplicationResultEntry( + ApplicationResult + .createFrom( + application))); + }); + }) + .handleAsync( + (ignored, error) -> { + if (error != null) { + fatalErrorHandler.onFatalError( + new FlinkException( + String.format( + "The application %s couldn't be marked as pre-cleanup finished in ApplicationResultStore.", + applicationId), + error)); + } + applicationCreateDirtyResultFutures.get(applicationId).complete(null); + return null; + }, + getMainThreadExecutor()); + } + + private CompletableFuture removeApplication( + ApplicationID applicationId, Collection jobs) { + return applicationResourceCleaner + .cleanupAsync(applicationId) + .thenCombine( + FutureUtils.waitForAll( + jobs.stream() + .map(jobMarkResultCleanFutures::get) + .collect(Collectors.toList())), + (unused1, unused2) -> + applicationResultStore.markResultAsCleanAsync(applicationId)) + .handle( + (ignored, t) -> { + if (t == null) { + log.debug( + "Cleanup for the application {} has finished. Application has been marked as clean.", + applicationId); + } else { + log.warn( + "Could not properly mark application {} result as clean.", + applicationId, + t); + } + return null; + }) + .thenRunAsync( + () -> applicationTerminationFutures.get(applicationId).complete(null), + getMainThreadExecutor()); + } + @VisibleForTesting Map getApplications() { return applications; @@ -955,7 +1207,6 @@ private void associateJobWithApplication(JobID jobId, ApplicationID applicationI checkState(applications.containsKey(applicationId)); applications.get(applicationId).addJob(jobId); - jobIdsToApplicationIds.put(jobId, applicationId); } private CompletableFuture handleTermination( @@ -1051,7 +1302,9 @@ private void runJob( final JobID jobId = jobManagerRunner.getJobID(); - partialExecutionGraphInfoStore.put(jobId, new CompletableFuture<>()); + jobCreateDirtyResultFutures.put(jobId, new CompletableFuture<>()); + jobMarkResultCleanFutures.put(jobId, new CompletableFuture<>()); + jobIdsToApplicationIds.put(jobId, applicationId); if (associateJobWithApplication) { associateJobWithApplication(jobId, applicationId); @@ -1314,22 +1567,7 @@ private Collection getCompletedJobDetails() { } private Stream getPartialExecutionGraphInfo() { - return partialExecutionGraphInfoStore.values().stream() - .filter(CompletableFuture::isDone) - .map( - executionGraphInfoFuture -> { - try { - ExecutionGraphInfo executionGraphInfo = - executionGraphInfoFuture.getNow(null); - if (executionGraphInfo != null) { - return executionGraphInfo; - } - } catch (Exception e) { - log.error("ExecutionGraphInfo future completed with exception", e); - } - return null; - }) - .filter(Objects::nonNull); + return partialExecutionGraphInfoStore.values().stream(); } @Override @@ -1453,17 +1691,9 @@ private ExecutionGraphInfo getExecutionGraphInfoFromStore(Throwable t, JobID job } private Optional getExecutionGraphInfoFromStore(JobID jobId) { - final CompletableFuture executionGraphInfoFuture = - partialExecutionGraphInfoStore.get(jobId); - if (executionGraphInfoFuture != null && executionGraphInfoFuture.isDone()) { - try { - ExecutionGraphInfo executionGraphInfo = executionGraphInfoFuture.getNow(null); - if (executionGraphInfo != null) { - return Optional.of(executionGraphInfo); - } - } catch (Exception e) { - log.error("Error while retrieving ExecutionGraphInfo for job {}", jobId, e); - } + final ExecutionGraphInfo executionGraphInfo = partialExecutionGraphInfoStore.get(jobId); + if (executionGraphInfo != null) { + return Optional.of(executionGraphInfo); } // check whether the job belongs to a completed application @@ -1850,19 +2080,14 @@ void registerJobManagerRunnerTerminationFuture( private CompletableFuture removeJob(JobID jobId, CleanupJobState cleanupJobState) { if (cleanupJobState.isGlobalCleanup()) { final ApplicationID applicationId = jobIdsToApplicationIds.get(jobId); - final CompletableFuture applicationTerminationFuture; - if (applicationTerminationFutures.containsKey(applicationId)) { - applicationTerminationFuture = applicationTerminationFutures.get(applicationId); - } else { - // This can occur during cleanup for recovered dirty job results in session mode - applicationTerminationFuture = CompletableFuture.completedFuture(null); - } + final CompletableFuture applicationCreateDirtyResultFuture = + applicationCreateDirtyResultFutures.get(applicationId); - // wait for the application termination before marking the job result as clean + // wait for the application dirty result creation before marking the job result as clean return globalResourceCleaner .cleanupAsync(jobId) .thenCombine( - applicationTerminationFuture, + applicationCreateDirtyResultFuture, (unused1, unused2) -> jobResultStore.markResultAsCleanAsync(jobId)) .handle( (unusedVoid, e) -> { @@ -1870,11 +2095,13 @@ private CompletableFuture removeJob(JobID jobId, CleanupJobState cleanupJo log.debug( "Cleanup for the job '{}' has finished. Job has been marked as clean.", jobId); + jobMarkResultCleanFutures.get(jobId).complete(null); } else { log.warn( "Could not properly mark job {} result as clean.", jobId, e); + jobMarkResultCleanFutures.get(jobId).completeExceptionally(e); } return null; }) @@ -1927,6 +2154,7 @@ private void terminateRunningApplications() { for (ApplicationID applicationId : applicationsToRemove) { applications.get(applicationId).dispose(); + applicationCreateDirtyResultFutures.get(applicationId).cancel(false); applicationTerminationFutures.get(applicationId).cancel(false); } } @@ -2013,6 +2241,7 @@ private CompletableFuture registerGloballyTerminatedJobInJobRes executionGraphInfo.getJobId()), error)); } + jobCreateDirtyResultFutures.get(jobId).complete(null); return CleanupJobState.globalCleanup(terminalJobStatus); }, getMainThreadExecutor(jobId)); @@ -2070,19 +2299,7 @@ private CompletableFuture createDirtyJobResultEntryAsync( * @param executionGraphInfo the execution graph information to be stored temporarily */ private void writeToExecutionGraphInfoStore(ExecutionGraphInfo executionGraphInfo) { - final JobID jobId = executionGraphInfo.getJobId(); - if (!partialExecutionGraphInfoStore.containsKey(jobId)) { - // This can only occur in tests for job termination without submitting the job - final CompletableFuture executionGraphInfoFuture = - new CompletableFuture<>(); - executionGraphInfoFuture.complete(executionGraphInfo); - partialExecutionGraphInfoStore.put(jobId, executionGraphInfoFuture); - return; - } - - partialExecutionGraphInfoStore - .get(executionGraphInfo.getJobId()) - .complete(executionGraphInfo); + partialExecutionGraphInfoStore.put(executionGraphInfo.getJobId(), executionGraphInfo); } private CompletableFuture archiveExecutionGraphToHistoryServer( 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 b7cc00e2a77d6..33a38deb58082 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 @@ -18,6 +18,8 @@ package org.apache.flink.runtime.dispatcher; +import org.apache.flink.runtime.application.AbstractApplication; +import org.apache.flink.runtime.highavailability.ApplicationResult; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.streaming.api.graph.ExecutionPlan; @@ -33,8 +35,10 @@ Dispatcher createDispatcher( DispatcherId fencingToken, Collection recoveredJobs, Collection recoveredDirtyJobResults, + Collection recoveredApplications, + Collection recoveredDirtyApplicationResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, - PartialDispatcherServicesWithJobPersistenceComponents - partialDispatcherServicesWithJobPersistenceComponents) + PartialDispatcherServicesWithPersistenceComponents + partialDispatcherServicesWithPersistenceComponents) 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 a4faf1300fe16..629a0c8d2fc54 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 @@ -23,8 +23,10 @@ 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.ApplicationResultStore; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.jobmanager.ApplicationWriter; import org.apache.flink.runtime.jobmanager.ExecutionPlanWriter; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; @@ -66,6 +68,10 @@ public class DispatcherServices { private final JobResultStore jobResultStore; + private final ApplicationWriter applicationWriter; + + private final ApplicationResultStore applicationResultStore; + private final JobManagerRunnerFactory jobManagerRunnerFactory; private final CleanupRunnerFactory cleanupRunnerFactory; @@ -88,6 +94,8 @@ public class DispatcherServices { JobManagerMetricGroup jobManagerMetricGroup, ExecutionPlanWriter planWriter, JobResultStore jobResultStore, + ApplicationWriter applicationWriter, + ApplicationResultStore applicationResultStore, JobManagerRunnerFactory jobManagerRunnerFactory, CleanupRunnerFactory cleanupRunnerFactory, Executor ioExecutor, @@ -111,6 +119,9 @@ public class DispatcherServices { Preconditions.checkNotNull(jobManagerMetricGroup, "JobManagerMetricGroup"); this.executionPlanWriter = Preconditions.checkNotNull(planWriter, "ExecutionPlanWriter"); this.jobResultStore = Preconditions.checkNotNull(jobResultStore, "JobResultStore"); + this.applicationWriter = Preconditions.checkNotNull(applicationWriter, "ApplicationWriter"); + this.applicationResultStore = + Preconditions.checkNotNull(applicationResultStore, "ApplicationResultStore"); this.jobManagerRunnerFactory = Preconditions.checkNotNull(jobManagerRunnerFactory, "JobManagerRunnerFactory"); this.cleanupRunnerFactory = @@ -172,6 +183,14 @@ public JobResultStore getJobResultStore() { return jobResultStore; } + public ApplicationWriter getApplicationWriter() { + return applicationWriter; + } + + public ApplicationResultStore getApplicationResultStore() { + return applicationResultStore; + } + JobManagerRunnerFactory getJobManagerRunnerFactory() { return jobManagerRunnerFactory; } @@ -189,31 +208,32 @@ public Collection getFailureEnrichers() { } public static DispatcherServices from( - PartialDispatcherServicesWithJobPersistenceComponents - partialDispatcherServicesWithJobPersistenceComponents, + PartialDispatcherServicesWithPersistenceComponents + partialDispatcherServicesWithPersistenceComponents, JobManagerRunnerFactory jobManagerRunnerFactory, CleanupRunnerFactory cleanupRunnerFactory) { return new DispatcherServices( - partialDispatcherServicesWithJobPersistenceComponents.getConfiguration(), - partialDispatcherServicesWithJobPersistenceComponents.getHighAvailabilityServices(), - partialDispatcherServicesWithJobPersistenceComponents + partialDispatcherServicesWithPersistenceComponents.getConfiguration(), + partialDispatcherServicesWithPersistenceComponents.getHighAvailabilityServices(), + partialDispatcherServicesWithPersistenceComponents .getResourceManagerGatewayRetriever(), - partialDispatcherServicesWithJobPersistenceComponents.getBlobServer(), - partialDispatcherServicesWithJobPersistenceComponents.getHeartbeatServices(), - partialDispatcherServicesWithJobPersistenceComponents.getArchivedApplicationStore(), - partialDispatcherServicesWithJobPersistenceComponents.getFatalErrorHandler(), - partialDispatcherServicesWithJobPersistenceComponents.getHistoryServerArchivist(), - partialDispatcherServicesWithJobPersistenceComponents - .getMetricQueryServiceAddress(), - partialDispatcherServicesWithJobPersistenceComponents.getOperationCaches(), - partialDispatcherServicesWithJobPersistenceComponents + partialDispatcherServicesWithPersistenceComponents.getBlobServer(), + partialDispatcherServicesWithPersistenceComponents.getHeartbeatServices(), + partialDispatcherServicesWithPersistenceComponents.getArchivedApplicationStore(), + partialDispatcherServicesWithPersistenceComponents.getFatalErrorHandler(), + partialDispatcherServicesWithPersistenceComponents.getHistoryServerArchivist(), + partialDispatcherServicesWithPersistenceComponents.getMetricQueryServiceAddress(), + partialDispatcherServicesWithPersistenceComponents.getOperationCaches(), + partialDispatcherServicesWithPersistenceComponents .getJobManagerMetricGroupFactory() .create(), - partialDispatcherServicesWithJobPersistenceComponents.getExecutionPlanWriter(), - partialDispatcherServicesWithJobPersistenceComponents.getJobResultStore(), + partialDispatcherServicesWithPersistenceComponents.getExecutionPlanWriter(), + partialDispatcherServicesWithPersistenceComponents.getJobResultStore(), + partialDispatcherServicesWithPersistenceComponents.getApplicationWriter(), + partialDispatcherServicesWithPersistenceComponents.getApplicationResultStore(), jobManagerRunnerFactory, cleanupRunnerFactory, - partialDispatcherServicesWithJobPersistenceComponents.getIoExecutor(), - partialDispatcherServicesWithJobPersistenceComponents.getFailureEnrichers()); + partialDispatcherServicesWithPersistenceComponents.getIoExecutor(), + partialDispatcherServicesWithPersistenceComponents.getFailureEnrichers()); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobPersistenceComponents.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithPersistenceComponents.java similarity index 77% rename from flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobPersistenceComponents.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithPersistenceComponents.java index 4f701bbffd0cc..3dc05b0881884 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithJobPersistenceComponents.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/PartialDispatcherServicesWithPersistenceComponents.java @@ -22,8 +22,10 @@ import org.apache.flink.core.failure.FailureEnricher; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.jobmanager.ApplicationWriter; import org.apache.flink.runtime.jobmanager.ExecutionPlanWriter; import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -35,13 +37,14 @@ import java.util.concurrent.Executor; /** {@link DispatcherFactory} services container. */ -public class PartialDispatcherServicesWithJobPersistenceComponents - extends PartialDispatcherServices { +public class PartialDispatcherServicesWithPersistenceComponents extends PartialDispatcherServices { private final ExecutionPlanWriter executionPlanWriter; private final JobResultStore jobResultStore; + private final ApplicationWriter applicationWriter; + private final ApplicationResultStore applicationResultStore; - private PartialDispatcherServicesWithJobPersistenceComponents( + private PartialDispatcherServicesWithPersistenceComponents( Configuration configuration, HighAvailabilityServices highAvailabilityServices, GatewayRetriever resourceManagerGatewayRetriever, @@ -56,7 +59,9 @@ private PartialDispatcherServicesWithJobPersistenceComponents( DispatcherOperationCaches operationCaches, Collection failureEnrichers, ExecutionPlanWriter executionPlanWriter, - JobResultStore jobResultStore) { + JobResultStore jobResultStore, + ApplicationWriter applicationWriter, + ApplicationResultStore applicationResultStore) { super( configuration, highAvailabilityServices, @@ -73,6 +78,8 @@ private PartialDispatcherServicesWithJobPersistenceComponents( failureEnrichers); this.executionPlanWriter = executionPlanWriter; this.jobResultStore = jobResultStore; + this.applicationWriter = applicationWriter; + this.applicationResultStore = applicationResultStore; } public ExecutionPlanWriter getExecutionPlanWriter() { @@ -83,11 +90,21 @@ public JobResultStore getJobResultStore() { return jobResultStore; } - public static PartialDispatcherServicesWithJobPersistenceComponents from( + public ApplicationWriter getApplicationWriter() { + return applicationWriter; + } + + public ApplicationResultStore getApplicationResultStore() { + return applicationResultStore; + } + + public static PartialDispatcherServicesWithPersistenceComponents from( PartialDispatcherServices partialDispatcherServices, ExecutionPlanWriter executionPlanWriter, - JobResultStore jobResultStore) { - return new PartialDispatcherServicesWithJobPersistenceComponents( + JobResultStore jobResultStore, + ApplicationWriter applicationWriter, + ApplicationResultStore applicationResultStore) { + return new PartialDispatcherServicesWithPersistenceComponents( partialDispatcherServices.getConfiguration(), partialDispatcherServices.getHighAvailabilityServices(), partialDispatcherServices.getResourceManagerGatewayRetriever(), @@ -102,6 +119,8 @@ public static PartialDispatcherServicesWithJobPersistenceComponents from( partialDispatcherServices.getOperationCaches(), partialDispatcherServices.getFailureEnrichers(), executionPlanWriter, - jobResultStore); + jobResultStore, + applicationWriter, + applicationResultStore); } } 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 239349c60f609..a11b912aa6661 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.application.AbstractApplication; import org.apache.flink.runtime.dispatcher.cleanup.CheckpointResourcesCleanupRunnerFactory; +import org.apache.flink.runtime.highavailability.ApplicationResult; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.streaming.api.graph.ExecutionPlan; @@ -35,9 +37,11 @@ public StandaloneDispatcher createDispatcher( DispatcherId fencingToken, Collection recoveredJobs, Collection recoveredDirtyJobResults, + Collection recoveredApplications, + Collection recoveredDirtyApplicationResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, - PartialDispatcherServicesWithJobPersistenceComponents - partialDispatcherServicesWithJobPersistenceComponents) + PartialDispatcherServicesWithPersistenceComponents + partialDispatcherServicesWithPersistenceComponents) throws Exception { // create the default dispatcher return new StandaloneDispatcher( @@ -45,9 +49,11 @@ public StandaloneDispatcher createDispatcher( fencingToken, recoveredJobs, recoveredDirtyJobResults, + recoveredApplications, + recoveredDirtyApplicationResults, dispatcherBootstrapFactory, DispatcherServices.from( - partialDispatcherServicesWithJobPersistenceComponents, + partialDispatcherServicesWithPersistenceComponents, JobMasterServiceLeadershipRunnerFactory.INSTANCE, CheckpointResourcesCleanupRunnerFactory.INSTANCE)); } 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 58b34f8bd972e..9abb133482c9f 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 @@ -18,6 +18,8 @@ package org.apache.flink.runtime.dispatcher; +import org.apache.flink.runtime.application.AbstractApplication; +import org.apache.flink.runtime.highavailability.ApplicationResult; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobmaster.JobMaster; import org.apache.flink.runtime.jobmaster.JobResult; @@ -37,6 +39,8 @@ public StandaloneDispatcher( DispatcherId fencingToken, Collection recoveredJobs, Collection recoveredDirtyJobResults, + Collection recoveredApplications, + Collection recoveredDirtyApplicationResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, DispatcherServices dispatcherServices) throws Exception { @@ -45,6 +49,8 @@ public StandaloneDispatcher( fencingToken, recoveredJobs, recoveredDirtyJobResults, + recoveredApplications, + recoveredDirtyApplicationResults, dispatcherBootstrapFactory, dispatcherServices); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ApplicationResourceCleaner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ApplicationResourceCleaner.java new file mode 100644 index 0000000000000..e2962e7ccb29c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ApplicationResourceCleaner.java @@ -0,0 +1,37 @@ +/* + * 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.ApplicationID; + +import java.util.concurrent.CompletableFuture; + +/** {@code ApplicationResourceCleaner} executes instances on the given {@code ApplicationID}. */ +@FunctionalInterface +public interface ApplicationResourceCleaner { + + /** + * Cleans application-related data from resources asynchronously. + * + * @param applicationId The {@link ApplicationID} referring to the application for which the + * data shall be cleaned up. + * @return the cleanup result future. + */ + CompletableFuture cleanupAsync(ApplicationID applicationId); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ApplicationResourceCleanerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ApplicationResourceCleanerFactory.java new file mode 100644 index 0000000000000..1266a86bb490a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/ApplicationResourceCleanerFactory.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.dispatcher.cleanup; + +import org.apache.flink.api.common.ApplicationID; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; + +import java.util.concurrent.Executor; + +/** + * {@code ApplicationResourceCleanerFactory} provides methods to create {@link + * ApplicationResourceCleaner} for application cleanup. + * + * @see GloballyCleanableApplicationResource + */ +public interface ApplicationResourceCleanerFactory { + + /** + * Creates {@link ApplicationResourceCleaner} that initiates {@link + * GloballyCleanableApplicationResource#globalCleanupAsync(ApplicationID, Executor)} calls. + * + * @param mainThreadExecutor Used for validating that the {@link + * GloballyCleanableApplicationResource#globalCleanupAsync(ApplicationID, Executor)} is + * called from the main thread. + */ + ApplicationResourceCleaner createApplicationResourceCleaner( + ComponentMainThreadExecutor mainThreadExecutor); +} 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 index 0c6b263ac14a7..b3128b4f654f6 100644 --- 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 @@ -218,17 +218,6 @@ private static JobStatus getJobStatus(JobResult jobResult) { private static ExecutionGraphInfo generateExecutionGraphInfo( JobResult jobResult, long initializationTimestamp) { return new ExecutionGraphInfo( - ArchivedExecutionGraph.createSparseArchivedExecutionGraph( - jobResult.getJobId(), - jobResult.getJobName(), - getJobStatus(jobResult), - null, - jobResult.getSerializedThrowable().orElse(null), - null, - jobResult.getStartTime() < 0 - ? initializationTimestamp - : jobResult.getStartTime(), - jobResult.getEndTime(), - jobResult.getApplicationId().orElse(null))); + ArchivedExecutionGraph.createFrom(jobResult, initializationTimestamp)); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultApplicationResourceCleaner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultApplicationResourceCleaner.java new file mode 100644 index 0000000000000..a19df4fd1ffa9 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultApplicationResourceCleaner.java @@ -0,0 +1,237 @@ +/* + * 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.api.common.ApplicationID; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.concurrent.RetryStrategy; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.stream.Collectors; + +/** + * {@code DefaultApplicationResourceCleaner} is the default implementation of {@link + * ApplicationResourceCleaner}. It will try to clean up any resource that was added. Failure will + * result in an individual retry of the cleanup. The overall cleanup result succeeds after all + * subtasks succeeded. + */ +public class DefaultApplicationResourceCleaner implements ApplicationResourceCleaner { + + private static final Logger LOG = + LoggerFactory.getLogger(DefaultApplicationResourceCleaner.class); + + private final ComponentMainThreadExecutor mainThreadExecutor; + private final Executor cleanupExecutor; + private final CleanupFn cleanupFn; + + private final Collection> regularCleanup; + + private final RetryStrategy retryStrategy; + + public static Builder forGloballyCleanableResources( + ComponentMainThreadExecutor mainThreadExecutor, + Executor cleanupExecutor, + RetryStrategy retryStrategy) { + return forCleanableResources( + mainThreadExecutor, + cleanupExecutor, + GloballyCleanableApplicationResource::globalCleanupAsync, + retryStrategy); + } + + @VisibleForTesting + static Builder forCleanableResources( + ComponentMainThreadExecutor mainThreadExecutor, + Executor cleanupExecutor, + CleanupFn cleanupFunction, + RetryStrategy retryStrategy) { + return new Builder<>(mainThreadExecutor, cleanupExecutor, cleanupFunction, retryStrategy); + } + + @VisibleForTesting + @FunctionalInterface + interface CleanupFn { + CompletableFuture cleanupAsync( + T resource, ApplicationID applicationId, Executor cleanupExecutor); + } + + /** + * {@code Builder} for creating {@code DefaultApplicationResourceCleaner} instances. + * + * @param The functional interface that's being translated into the internally used {@link + * CleanupFn}. + */ + public static class Builder { + + private final ComponentMainThreadExecutor mainThreadExecutor; + private final Executor cleanupExecutor; + private final CleanupFn cleanupFn; + + private final RetryStrategy retryStrategy; + + private final Collection> prioritizedCleanup = new ArrayList<>(); + private final Collection> regularCleanup = new ArrayList<>(); + + private Builder( + ComponentMainThreadExecutor mainThreadExecutor, + Executor cleanupExecutor, + CleanupFn cleanupFn, + RetryStrategy retryStrategy) { + this.mainThreadExecutor = mainThreadExecutor; + this.cleanupExecutor = cleanupExecutor; + this.cleanupFn = cleanupFn; + this.retryStrategy = retryStrategy; + } + + /** + * Prioritized cleanups run before their regular counterparts. This method enables the + * caller to model dependencies between cleanup tasks. The order in which cleanable + * resources are added matters, i.e. if two cleanable resources are added as prioritized + * cleanup tasks, the resource being added first will block the cleanup of the second + * resource. All prioritized cleanup resources will run and finish before any resource that + * is added using {@link #withRegularCleanup(String, Object)} is started. + * + * @param label The label being used when logging errors in the given cleanup. + * @param prioritizedCleanup The cleanup callback that is going to be prioritized. + */ + public Builder withPrioritizedCleanup(String label, T prioritizedCleanup) { + this.prioritizedCleanup.add(new CleanupWithLabel<>(prioritizedCleanup, label)); + return this; + } + + /** + * Regular cleanups are resources for which the cleanup is triggered after all prioritized + * cleanups succeeded. All added regular cleanups will run concurrently to each other. + * + * @param label The label being used when logging errors in the given cleanup. + * @param regularCleanup The cleanup callback that is going to run after all prioritized + * cleanups are finished. + * @see #withPrioritizedCleanup(String, Object) + */ + public Builder withRegularCleanup(String label, T regularCleanup) { + this.regularCleanup.add(new CleanupWithLabel<>(regularCleanup, label)); + return this; + } + + public DefaultApplicationResourceCleaner build() { + return new DefaultApplicationResourceCleaner<>( + mainThreadExecutor, + cleanupExecutor, + cleanupFn, + prioritizedCleanup, + regularCleanup, + retryStrategy); + } + } + + private DefaultApplicationResourceCleaner( + ComponentMainThreadExecutor mainThreadExecutor, + Executor cleanupExecutor, + CleanupFn cleanupFn, + Collection> prioritizedCleanup, + Collection> regularCleanup, + RetryStrategy retryStrategy) { + this.mainThreadExecutor = mainThreadExecutor; + this.cleanupExecutor = cleanupExecutor; + this.cleanupFn = cleanupFn; + this.regularCleanup = regularCleanup; + this.retryStrategy = retryStrategy; + } + + @Override + public CompletableFuture cleanupAsync(ApplicationID applicationId) { + mainThreadExecutor.assertRunningInMainThread(); + + CompletableFuture cleanupFuture = FutureUtils.completedVoidFuture(); + + return cleanupFuture.thenCompose( + ignoredValue -> + FutureUtils.completeAll( + regularCleanup.stream() + .map( + cleanupWithLabel -> + withRetry( + applicationId, + cleanupWithLabel.getLabel(), + cleanupWithLabel.getCleanup())) + .collect(Collectors.toList()))); + } + + private CompletableFuture withRetry( + ApplicationID applicationId, String label, T cleanup) { + return FutureUtils.retryWithDelay( + () -> + cleanupFn + .cleanupAsync(cleanup, applicationId, cleanupExecutor) + .whenComplete( + (value, throwable) -> { + if (throwable != null) { + final String logMessage = + String.format( + "Cleanup of %s failed for application %s due to a %s: %s", + label, + applicationId, + throwable + .getClass() + .getSimpleName(), + throwable.getMessage()); + if (LOG.isTraceEnabled()) { + LOG.warn(logMessage, throwable); + } else { + LOG.warn(logMessage); + } + } + }), + retryStrategy, + mainThreadExecutor); + } + + /** + * {@code CleanupWithLabel} makes it possible to attach a label to a given cleanup that can be + * used as human-readable representation of the corresponding cleanup. + * + * @param The type of cleanup. + */ + private static class CleanupWithLabel { + + private final CLEANUP_TYPE cleanup; + private final String label; + + public CleanupWithLabel(CLEANUP_TYPE cleanup, String label) { + this.cleanup = cleanup; + this.label = label; + } + + public CLEANUP_TYPE getCleanup() { + return cleanup; + } + + public String getLabel() { + return label; + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherApplicationResourceCleanerFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherApplicationResourceCleanerFactory.java new file mode 100644 index 0000000000000..570e1360cf8c4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherApplicationResourceCleanerFactory.java @@ -0,0 +1,78 @@ +/* + * 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.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.dispatcher.DispatcherServices; +import org.apache.flink.runtime.jobmanager.ApplicationWriter; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.concurrent.RetryStrategy; + +import java.util.concurrent.Executor; + +/** + * {@code DispatcherApplicationResourceCleanerFactory} instantiates {@link + * ApplicationResourceCleaner} instances that clean cleanable resources from the {@link + * org.apache.flink.runtime.dispatcher.Dispatcher}. + */ +public class DispatcherApplicationResourceCleanerFactory + implements ApplicationResourceCleanerFactory { + + private static final String APPLICATION_STORE_LABEL = "ApplicationStore"; + private static final String BLOB_SERVER_LABEL = "BlobServer"; + + private final Executor cleanupExecutor; + private final RetryStrategy retryStrategy; + + private final ApplicationWriter applicationWriter; + private final BlobServer blobServer; + + public DispatcherApplicationResourceCleanerFactory(DispatcherServices dispatcherServices) { + this( + dispatcherServices.getIoExecutor(), + CleanupRetryStrategyFactory.INSTANCE.createRetryStrategy( + dispatcherServices.getConfiguration()), + dispatcherServices.getApplicationWriter(), + dispatcherServices.getBlobServer()); + } + + @VisibleForTesting + public DispatcherApplicationResourceCleanerFactory( + Executor cleanupExecutor, + RetryStrategy retryStrategy, + ApplicationWriter applicationWriter, + BlobServer blobServer) { + this.cleanupExecutor = Preconditions.checkNotNull(cleanupExecutor); + this.retryStrategy = retryStrategy; + this.applicationWriter = Preconditions.checkNotNull(applicationWriter); + this.blobServer = Preconditions.checkNotNull(blobServer); + } + + @Override + public ApplicationResourceCleaner createApplicationResourceCleaner( + ComponentMainThreadExecutor mainThreadExecutor) { + return DefaultApplicationResourceCleaner.forGloballyCleanableResources( + mainThreadExecutor, cleanupExecutor, retryStrategy) + .withRegularCleanup(APPLICATION_STORE_LABEL, applicationWriter) + .withRegularCleanup(BLOB_SERVER_LABEL, blobServer) + .build(); + } +} 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 f84ca53baa64f..dbc181faaa05b 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 @@ -21,10 +21,14 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.application.AbstractApplication; 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.ApplicationResult; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.jobmanager.ApplicationStore; import org.apache.flink.runtime.jobmanager.ExecutionPlanWriter; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.FatalErrorHandler; @@ -260,8 +264,12 @@ DispatcherGatewayService create( DispatcherId dispatcherId, Collection recoveredJobs, Collection recoveredDirtyJobResults, + Collection recoveredApplications, + Collection recoveredDirtyApplicationResults, ExecutionPlanWriter executionPlanWriter, - JobResultStore jobResultStore); + JobResultStore jobResultStore, + ApplicationStore applicationStore, + ApplicationResultStore applicationResultStore); } /** 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 b0e2c712e4493..1767ae6b6d89b 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 @@ -18,13 +18,17 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.runtime.application.AbstractApplication; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherFactory; 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.PartialDispatcherServicesWithJobPersistenceComponents; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithPersistenceComponents; +import org.apache.flink.runtime.highavailability.ApplicationResult; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.jobmanager.ApplicationStore; import org.apache.flink.runtime.jobmanager.ExecutionPlanWriter; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rpc.RpcService; @@ -57,8 +61,12 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( DispatcherId fencingToken, Collection recoveredJobs, Collection recoveredDirtyJobResults, + Collection recoveredApplications, + Collection recoveredDirtyApplicationResults, ExecutionPlanWriter executionPlanWriter, - JobResultStore jobResultStore) { + JobResultStore jobResultStore, + ApplicationStore applicationStore, + ApplicationResultStore applicationResultStore) { final Dispatcher dispatcher; try { @@ -68,12 +76,16 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create( fencingToken, recoveredJobs, recoveredDirtyJobResults, + recoveredApplications, + recoveredDirtyApplicationResults, (dispatcherGateway, scheduledExecutor, errorHandler) -> new NoOpDispatcherBootstrap(), - PartialDispatcherServicesWithJobPersistenceComponents.from( + PartialDispatcherServicesWithPersistenceComponents.from( partialDispatcherServices, executionPlanWriter, - jobResultStore)); + jobResultStore, + applicationStore, + applicationResultStore)); } 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 fdd9333df4740..d0a2e4192172f 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 @@ -20,7 +20,7 @@ import org.apache.flink.runtime.dispatcher.DispatcherFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; +import org.apache.flink.runtime.jobmanager.PersistenceComponentFactory; import org.apache.flink.runtime.leaderelection.LeaderElection; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -43,7 +43,7 @@ public DefaultDispatcherRunnerFactory( public DispatcherRunner createDispatcherRunner( LeaderElection leaderElection, FatalErrorHandler fatalErrorHandler, - JobPersistenceComponentFactory jobPersistenceComponentFactory, + PersistenceComponentFactory persistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices) @@ -51,7 +51,7 @@ public DispatcherRunner createDispatcherRunner( final DispatcherLeaderProcessFactory dispatcherLeaderProcessFactory = dispatcherLeaderProcessFactoryFactory.createFactory( - jobPersistenceComponentFactory, + persistenceComponentFactory, 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 222c349870100..e630c066a3d0d 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.JobPersistenceComponentFactory; +import org.apache.flink.runtime.jobmanager.PersistenceComponentFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -31,7 +31,7 @@ public interface DispatcherLeaderProcessFactoryFactory { DispatcherLeaderProcessFactory createFactory( - JobPersistenceComponentFactory jobPersistenceComponentFactory, + PersistenceComponentFactory persistenceComponentFactory, 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 887ae8209e7a8..ed8f7900128cf 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.JobPersistenceComponentFactory; +import org.apache.flink.runtime.jobmanager.PersistenceComponentFactory; import org.apache.flink.runtime.leaderelection.LeaderElection; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; @@ -32,7 +32,7 @@ public interface DispatcherRunnerFactory { DispatcherRunner createDispatcherRunner( LeaderElection leaderElection, FatalErrorHandler fatalErrorHandler, - JobPersistenceComponentFactory jobPersistenceComponentFactory, + PersistenceComponentFactory persistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices) 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 a01c32f905881..22504c20235bc 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 @@ -18,16 +18,26 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.api.common.ApplicationID; import org.apache.flink.api.common.JobID; -import org.apache.flink.runtime.client.DuplicateJobSubmissionException; +import org.apache.flink.api.common.JobInfo; +import org.apache.flink.api.common.JobInfoImpl; +import org.apache.flink.configuration.RpcOptions; +import org.apache.flink.runtime.application.AbstractApplication; +import org.apache.flink.runtime.application.SingleJobApplication; +import org.apache.flink.runtime.blob.BlobServer; 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.ApplicationResult; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.jobmanager.ApplicationStore; +import org.apache.flink.runtime.jobmanager.ApplicationStoreEntry; import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.messages.FlinkApplicationNotFoundException; import org.apache.flink.runtime.rpc.FatalErrorHandler; -import org.apache.flink.runtime.rpc.RpcUtils; import org.apache.flink.streaming.api.graph.ExecutionPlan; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkRuntimeException; @@ -36,15 +46,19 @@ import org.apache.flink.util.function.FunctionUtils; import java.io.IOException; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; 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.function.Function; import java.util.stream.Collectors; /** @@ -60,6 +74,12 @@ public class SessionDispatcherLeaderProcess extends AbstractDispatcherLeaderProc private final JobResultStore jobResultStore; + private final ApplicationStore applicationStore; + + private final ApplicationResultStore applicationResultStore; + + private final BlobServer blobServer; + private final Executor ioExecutor; private CompletableFuture onGoingRecoveryOperation = FutureUtils.completedVoidFuture(); @@ -69,6 +89,9 @@ private SessionDispatcherLeaderProcess( DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, ExecutionPlanStore executionPlanStore, JobResultStore jobResultStore, + ApplicationStore applicationStore, + ApplicationResultStore applicationResultStore, + BlobServer blobServer, Executor ioExecutor, FatalErrorHandler fatalErrorHandler) { super(leaderSessionId, fatalErrorHandler); @@ -76,6 +99,9 @@ private SessionDispatcherLeaderProcess( this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; this.executionPlanStore = executionPlanStore; this.jobResultStore = jobResultStore; + this.applicationStore = applicationStore; + this.applicationResultStore = applicationResultStore; + this.blobServer = blobServer; this.ioExecutor = ioExecutor; } @@ -83,8 +109,7 @@ private SessionDispatcherLeaderProcess( protected void onStart() { startServices(); - onGoingRecoveryOperation = - createDispatcherBasedOnRecoveredExecutionPlansAndRecoveredDirtyJobResults(); + onGoingRecoveryOperation = createDispatcherBasedOnRecoveredApplicationsAndJobs(); } private void startServices() { @@ -98,46 +123,93 @@ private void startServices() { getClass().getSimpleName()), e); } + + try { + applicationStore.start(); + } catch (Exception e) { + throw new FlinkRuntimeException( + String.format( + "Could not start %s when trying to start the %s.", + applicationStore.getClass().getSimpleName(), + getClass().getSimpleName()), + e); + } } private void createDispatcherIfRunning( Collection executionPlans, - Collection recoveredDirtyJobResults) { + Collection recoveredDirtyJobResults, + Collection recoveredApplications, + Collection recoveredDirtyApplicationResults) { runIfStateIs( - State.RUNNING, () -> createDispatcher(executionPlans, recoveredDirtyJobResults)); + State.RUNNING, + () -> + createDispatcher( + executionPlans, + recoveredDirtyJobResults, + recoveredApplications, + recoveredDirtyApplicationResults)); } private void createDispatcher( Collection executionPlans, - Collection recoveredDirtyJobResults) { + Collection recoveredDirtyJobResults, + Collection recoveredApplications, + Collection recoveredDirtyApplicationResults) { final DispatcherGatewayService dispatcherService = dispatcherGatewayServiceFactory.create( DispatcherId.fromUuid(getLeaderSessionId()), executionPlans, recoveredDirtyJobResults, + recoveredApplications, + recoveredDirtyApplicationResults, executionPlanStore, - jobResultStore); + jobResultStore, + applicationStore, + applicationResultStore); completeDispatcherSetup(dispatcherService); } - private CompletableFuture - createDispatcherBasedOnRecoveredExecutionPlansAndRecoveredDirtyJobResults() { - // TODO support application recovery which may require fetching user jar from blob server - - final CompletableFuture> dirtyJobsFuture = + private CompletableFuture createDispatcherBasedOnRecoveredApplicationsAndJobs() { + final CompletableFuture> dirtyJobResultsFuture = CompletableFuture.supplyAsync(this::getDirtyJobResultsIfRunning, ioExecutor); - return dirtyJobsFuture - .thenApplyAsync( - dirtyJobs -> - this.recoverJobsIfRunning( - dirtyJobs.stream() + final CompletableFuture> recoveredJobsFuture = + dirtyJobResultsFuture.thenApplyAsync( + dirtyJobResults -> + recoverJobsIfRunning( + dirtyJobResults.stream() .map(JobResult::getJobId) .collect(Collectors.toSet())), - ioExecutor) - .thenAcceptBoth(dirtyJobsFuture, this::createDispatcherIfRunning) + ioExecutor); + + final CompletableFuture> dirtyApplicationResultsFuture = + CompletableFuture.supplyAsync( + this::getDirtyApplicationResultsIfRunning, ioExecutor); + + final CompletableFuture> recoveredApplicationsFuture = + dirtyApplicationResultsFuture.thenCombineAsync( + recoveredJobsFuture, + (dirtyApplicationResults, recoveredJobs) -> { + return recoverApplicationsIfRunning( + dirtyApplicationResults.stream() + .map(ApplicationResult::getApplicationId) + .collect(Collectors.toSet()), + recoveredJobsFuture.join(), + dirtyJobResultsFuture.join()); + }, + ioExecutor); + + return recoveredApplicationsFuture + .thenRun( + () -> + createDispatcherIfRunning( + recoveredJobsFuture.join(), + dirtyJobResultsFuture.join(), + recoveredApplicationsFuture.join(), + dirtyApplicationResultsFuture.join())) .handle(this::onErrorIfRunning); } @@ -205,6 +277,128 @@ private Collection getDirtyJobResults() { } } + private Collection getDirtyApplicationResultsIfRunning() { + return supplyUnsynchronizedIfRunning(this::getDirtyApplicationResults) + .orElse(Collections.emptyList()); + } + + private Collection getDirtyApplicationResults() { + try { + return applicationResultStore.getDirtyResults(); + } catch (IOException e) { + throw new FlinkRuntimeException( + "Could not retrieve ApplicationResults from ApplicationResultStore", e); + } + } + + private Collection recoverApplicationsIfRunning( + Set recoveredDirtyApplicationResults, + Collection recoveredJobs, + Collection dirtyJobResults) { + return supplyUnsynchronizedIfRunning( + () -> + recoverApplications( + recoveredDirtyApplicationResults, + recoveredJobs, + dirtyJobResults)) + .orElse(Collections.emptyList()); + } + + private Collection recoverApplications( + Set recoveredDirtyJobResults, + Collection recoveredJobs, + Collection dirtyJobResults) { + log.info("Recover all persisted applications that are not finished, yet."); + final Collection applicationIds = getApplicationIds(); + final Collection recoveredApplications = new ArrayList<>(); + + final Map> recoveredJobInfosByApplication = + new HashMap<>(); + for (ExecutionPlan executionPlan : recoveredJobs) { + ApplicationID applicationId = + executionPlan + .getApplicationId() + .orElseThrow( + () -> + new IllegalStateException( + "Application ID is missing in the recovered execution plan. This suggests the job was submitted through an unsupported or incomplete path.")); + recoveredJobInfosByApplication + .computeIfAbsent(applicationId, k -> new ArrayList<>()) + .add(new JobInfoImpl(executionPlan.getJobID(), executionPlan.getName())); + } + + final Map> recoveredTerminalJobInfosByApplication = + new HashMap<>(); + for (JobResult jobResult : dirtyJobResults) { + ApplicationID applicationId = + jobResult + .getApplicationId() + .orElseThrow( + () -> + new IllegalStateException( + "Application ID is missing in the recovered job result. This suggests the job was submitted through an unsupported or incomplete path.")); + recoveredTerminalJobInfosByApplication + .computeIfAbsent(applicationId, k -> new ArrayList<>()) + .add(new JobInfoImpl(jobResult.getJobId(), jobResult.getJobName())); + } + + for (ApplicationID applicationId : applicationIds) { + if (!recoveredDirtyJobResults.contains(applicationId)) { + Collection recoveredJobInfos = + recoveredJobInfosByApplication.getOrDefault( + applicationId, Collections.emptyList()); + Collection recoveredTerminalJobInfos = + recoveredTerminalJobInfosByApplication.getOrDefault( + applicationId, Collections.emptyList()); + tryRecoverApplication(applicationId, recoveredJobInfos, recoveredTerminalJobInfos) + .ifPresent(recoveredApplications::add); + } else { + log.info( + "Skipping recovery of an application with id {}, because it already reached a globally terminal state", + applicationId); + } + } + + log.info("Successfully recovered {} persisted applications.", recoveredApplications.size()); + + return recoveredApplications; + } + + private Collection getApplicationIds() { + try { + return applicationStore.getApplicationIds(); + } catch (Exception e) { + throw new FlinkRuntimeException( + "Could not retrieve application ids of persisted applications.", e); + } + } + + private Optional tryRecoverApplication( + ApplicationID applicationId, + Collection recoveredJobInfos, + Collection recoveredTerminalJobInfos) { + log.info("Trying to recover application with id {}.", applicationId); + try { + Optional applicationStoreEntry = + applicationStore.recoverApplication(applicationId); + if (applicationStoreEntry.isEmpty()) { + log.info( + "Skipping recovery of application with id {}, because it already finished in a previous execution", + applicationId); + return Optional.empty(); + } + + return Optional.of( + applicationStoreEntry + .get() + .getApplication( + blobServer, recoveredJobInfos, recoveredTerminalJobInfos)); + } catch (Exception e) { + throw new FlinkRuntimeException( + String.format("Could not recover application with id %s.", applicationId), e); + } + } + @Override protected CompletableFuture onClose() { return CompletableFuture.runAsync(this::stopServices, ioExecutor); @@ -213,6 +407,7 @@ protected CompletableFuture onClose() { private void stopServices() { try { executionPlanStore.stop(); + applicationStore.stop(); } catch (Exception e) { ExceptionUtils.rethrow(e); } @@ -251,28 +446,40 @@ private Optional> submitAddedJobIfRunning(ExecutionPlan private CompletableFuture submitAddedJob(ExecutionPlan executionPlan) { final DispatcherGateway dispatcherGateway = getDispatcherGatewayInternal(); + final JobID jobId = executionPlan.getJobID(); + final Duration timeout = + executionPlan.getJobConfiguration().get(RpcOptions.ASK_TIMEOUT_DURATION); + // Skip job submission if its associated application exists, as the application will handle + // the job recovery + ApplicationID applicationId = executionPlan.getApplicationId().orElse(null); return dispatcherGateway - .submitJob(executionPlan, RpcUtils.INF_TIMEOUT) - .thenApply(FunctionUtils.nullFn()) - .exceptionally(this::filterOutDuplicateJobSubmissionException); - } - - private Void filterOutDuplicateJobSubmissionException(Throwable throwable) { - final Throwable strippedException = ExceptionUtils.stripCompletionException(throwable); - if (strippedException instanceof DuplicateJobSubmissionException) { - final DuplicateJobSubmissionException duplicateJobSubmissionException = - (DuplicateJobSubmissionException) strippedException; - - log.debug( - "Ignore recovered job {} because the job is currently being executed.", - duplicateJobSubmissionException.getJobID(), - duplicateJobSubmissionException); - - return null; - } else { - throw new CompletionException(throwable); - } + .requestApplication(applicationId, timeout) + .handle( + (ignored, throwable) -> { + if (throwable == null) { + log.debug( + "Ignore recovered job {} because the associated application {} exists.", + jobId, + applicationId); + return FutureUtils.completedVoidFuture(); + } + + final Throwable strippedException = + ExceptionUtils.stripCompletionException(throwable); + if (strippedException instanceof FlinkApplicationNotFoundException) { + log.info( + "Submitting job {} added to the {} by another process by wrapping it in a SingleJobApplication", + jobId, + executionPlanStore.getClass().getSimpleName()); + return dispatcherGateway + .submitApplication( + new SingleJobApplication(executionPlan), timeout) + .thenApply(FunctionUtils.nullFn()); + } + throw new CompletionException(throwable); + }) + .thenCompose(Function.identity()); } private DispatcherGateway getDispatcherGatewayInternal() { @@ -322,6 +529,9 @@ public static SessionDispatcherLeaderProcess create( DispatcherGatewayServiceFactory dispatcherFactory, ExecutionPlanStore executionPlanStore, JobResultStore jobResultStore, + ApplicationStore applicationStore, + ApplicationResultStore applicationResultStore, + BlobServer blobServer, Executor ioExecutor, FatalErrorHandler fatalErrorHandler) { return new SessionDispatcherLeaderProcess( @@ -329,6 +539,9 @@ public static SessionDispatcherLeaderProcess create( dispatcherFactory, executionPlanStore, jobResultStore, + applicationStore, + applicationResultStore, + blobServer, 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 a239746b799da..777c66e0f5aab 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,8 @@ package org.apache.flink.runtime.dispatcher.runner; import org.apache.flink.annotation.Internal; -import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.jobmanager.PersistenceComponentFactory; import org.apache.flink.runtime.rpc.FatalErrorHandler; import java.util.UUID; @@ -31,18 +32,21 @@ public class SessionDispatcherLeaderProcessFactory implements DispatcherLeaderPr private final AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory; - private final JobPersistenceComponentFactory jobPersistenceComponentFactory; + private final PersistenceComponentFactory persistenceComponentFactory; + private final BlobServer blobServer; private final Executor ioExecutor; private final FatalErrorHandler fatalErrorHandler; public SessionDispatcherLeaderProcessFactory( AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherGatewayServiceFactory, - JobPersistenceComponentFactory jobPersistenceComponentFactory, + PersistenceComponentFactory persistenceComponentFactory, + BlobServer blobServer, Executor ioExecutor, FatalErrorHandler fatalErrorHandler) { this.dispatcherGatewayServiceFactory = dispatcherGatewayServiceFactory; - this.jobPersistenceComponentFactory = jobPersistenceComponentFactory; + this.persistenceComponentFactory = persistenceComponentFactory; + this.blobServer = blobServer; this.ioExecutor = ioExecutor; this.fatalErrorHandler = fatalErrorHandler; } @@ -52,8 +56,11 @@ public DispatcherLeaderProcess create(UUID leaderSessionID) { return SessionDispatcherLeaderProcess.create( leaderSessionID, dispatcherGatewayServiceFactory, - jobPersistenceComponentFactory.createExecutionPlanStore(), - jobPersistenceComponentFactory.createJobResultStore(), + persistenceComponentFactory.createExecutionPlanStore(), + persistenceComponentFactory.createJobResultStore(), + persistenceComponentFactory.createApplicationStore(), + persistenceComponentFactory.createApplicationResultStore(), + blobServer, 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 ee2fb081647fe..99fb4667eec3d 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.JobPersistenceComponentFactory; +import org.apache.flink.runtime.jobmanager.PersistenceComponentFactory; 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( - JobPersistenceComponentFactory jobPersistenceComponentFactory, + PersistenceComponentFactory persistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices, @@ -50,7 +50,8 @@ public DispatcherLeaderProcessFactory createFactory( return new SessionDispatcherLeaderProcessFactory( dispatcherGatewayServiceFactory, - jobPersistenceComponentFactory, + persistenceComponentFactory, + partialDispatcherServices.getBlobServer(), 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 0408c4af5c301..60abff3464afd 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 @@ -37,7 +37,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.HaServicesJobPersistenceComponentFactory; +import org.apache.flink.runtime.jobmanager.HaServicesPersistenceComponentFactory; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.metrics.MetricRegistry; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; @@ -226,7 +226,7 @@ public DispatcherResourceManagerComponent create( dispatcherRunnerFactory.createDispatcherRunner( highAvailabilityServices.getDispatcherLeaderElection(), fatalErrorHandler, - new HaServicesJobPersistenceComponentFactory(highAvailabilityServices), + new HaServicesPersistenceComponentFactory(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 e259385507fab..da553564a3640 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 @@ -32,6 +32,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; +import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rest.messages.JobPlanInfo; import org.apache.flink.runtime.scheduler.VertexParallelismInformation; import org.apache.flink.runtime.scheduler.VertexParallelismStore; @@ -401,6 +402,28 @@ public static ArchivedExecutionGraph createFrom( executionGraph.getApplicationId().orElse(null)); } + /** + * Create a {@link ArchivedExecutionGraph} from the given {@link JobResult}. + * + * @param jobResult to create the ArchivedExecutionGraph from + * @param initializationTimestamp optionally overrides the initialization timestamp if the + * jobResult does not have a valid one + * @return ArchivedExecutionGraph created from the given jobResult + */ + public static ArchivedExecutionGraph createFrom( + JobResult jobResult, long initializationTimestamp) { + return createSparseArchivedExecutionGraph( + jobResult.getJobId(), + jobResult.getJobName(), + jobResult.getJobStatus().orElseThrow(), + null, + jobResult.getSerializedThrowable().orElse(null), + null, + jobResult.getStartTime() < 0 ? initializationTimestamp : jobResult.getStartTime(), + jobResult.getEndTime(), + jobResult.getApplicationId().orElse(null)); + } + /** * Create a sparse ArchivedExecutionGraph for a job. Most fields will be empty, only job status * and error-related fields are set. 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 24201fa28205a..1c0a416f27394 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 @@ -23,6 +23,7 @@ import org.apache.flink.runtime.blob.BlobStore; import org.apache.flink.runtime.blob.BlobStoreService; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.jobmanager.ApplicationStore; import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; import org.apache.flink.runtime.leaderelection.DefaultLeaderElectionService; import org.apache.flink.runtime.leaderelection.LeaderElection; @@ -67,6 +68,8 @@ public abstract class AbstractHaServices implements HighAvailabilityServices { private final JobResultStore jobResultStore; + private final ApplicationResultStore applicationResultStore; + private final DefaultLeaderElectionService leaderElectionService; protected AbstractHaServices( @@ -74,12 +77,14 @@ protected AbstractHaServices( LeaderElectionDriverFactory driverFactory, Executor ioExecutor, BlobStoreService blobStoreService, - JobResultStore jobResultStore) { + JobResultStore jobResultStore, + ApplicationResultStore applicationResultStore) { this.configuration = checkNotNull(config); this.ioExecutor = checkNotNull(ioExecutor); this.blobStoreService = checkNotNull(blobStoreService); this.jobResultStore = checkNotNull(jobResultStore); + this.applicationResultStore = checkNotNull(applicationResultStore); this.leaderElectionService = new DefaultLeaderElectionService(driverFactory); } @@ -145,6 +150,16 @@ public JobResultStore getJobResultStore() throws Exception { return jobResultStore; } + @Override + public ApplicationStore getApplicationStore() throws Exception { + return createApplicationStore(); + } + + @Override + public ApplicationResultStore getApplicationResultStore() throws Exception { + return applicationResultStore; + } + @Override public BlobStore createBlobStore() { return blobStoreService; @@ -248,6 +263,14 @@ public CompletableFuture globalCleanupAsync(JobID jobID, Executor executor */ protected abstract ExecutionPlanStore createExecutionPlanStore() throws Exception; + /** + * Create the submitted application store for the job manager. + * + * @return Submitted application store + * @throws Exception if the submitted application store could not be created + */ + protected abstract ApplicationStore createApplicationStore() throws Exception; + /** * 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/AbstractThreadsafeApplicationResultStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeApplicationResultStore.java new file mode 100644 index 0000000000000..5fbdacff80707 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/AbstractThreadsafeApplicationResultStore.java @@ -0,0 +1,161 @@ +/* + * 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.ApplicationID; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.function.SupplierWithException; +import org.apache.flink.util.function.ThrowingRunnable; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.GuardedBy; + +import java.io.IOException; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** An abstract class for threadsafe implementations of the {@link ApplicationResultStore}. */ +public abstract class AbstractThreadsafeApplicationResultStore implements ApplicationResultStore { + + private static final Logger LOG = + LoggerFactory.getLogger(AbstractThreadsafeApplicationResultStore.class); + + private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); + + private final Executor ioExecutor; + + protected AbstractThreadsafeApplicationResultStore(Executor ioExecutor) { + this.ioExecutor = ioExecutor; + } + + @Override + public CompletableFuture createDirtyResultAsync( + ApplicationResultEntry applicationResultEntry) { + return hasApplicationResultEntryAsync(applicationResultEntry.getApplicationId()) + .thenAccept( + hasApplicationResultEntry -> + Preconditions.checkState( + !hasApplicationResultEntry, + "Application result store already contains an entry for application %s", + applicationResultEntry.getApplicationId())) + .thenCompose( + ignoredVoid -> + withWriteLockAsync( + () -> createDirtyResultInternal(applicationResultEntry))); + } + + @GuardedBy("readWriteLock") + protected abstract void createDirtyResultInternal(ApplicationResultEntry applicationResultEntry) + throws IOException; + + @Override + public CompletableFuture markResultAsCleanAsync(ApplicationID applicationId) { + return hasCleanApplicationResultEntryAsync(applicationId) + .thenCompose( + hasCleanApplicationResultEntry -> { + if (hasCleanApplicationResultEntry) { + LOG.debug( + "The application {} is already marked as clean. No action required.", + applicationId); + return FutureUtils.completedVoidFuture(); + } + + return withWriteLockAsync( + () -> markResultAsCleanInternal(applicationId)); + }); + } + + @GuardedBy("readWriteLock") + protected abstract void markResultAsCleanInternal(ApplicationID applicationId) + throws IOException, NoSuchElementException; + + @Override + public CompletableFuture hasApplicationResultEntryAsync(ApplicationID applicationId) { + return withReadLockAsync( + () -> + hasDirtyApplicationResultEntryInternal(applicationId) + || hasCleanApplicationResultEntryInternal(applicationId)); + } + + @Override + public CompletableFuture hasDirtyApplicationResultEntryAsync( + ApplicationID applicationId) { + return withReadLockAsync(() -> hasDirtyApplicationResultEntryInternal(applicationId)); + } + + @GuardedBy("readWriteLock") + protected abstract boolean hasDirtyApplicationResultEntryInternal(ApplicationID applicationId) + throws IOException; + + @Override + public CompletableFuture hasCleanApplicationResultEntryAsync( + ApplicationID applicationId) { + return withReadLockAsync(() -> hasCleanApplicationResultEntryInternal(applicationId)); + } + + @GuardedBy("readWriteLock") + protected abstract boolean hasCleanApplicationResultEntryInternal(ApplicationID applicationId) + throws IOException; + + @Override + public Set getDirtyResults() throws IOException { + return withReadLock(this::getDirtyResultsInternal); + } + + @GuardedBy("readWriteLock") + protected abstract Set getDirtyResultsInternal() throws IOException; + + private CompletableFuture withWriteLockAsync(ThrowingRunnable runnable) { + return FutureUtils.runAsync( + () -> { + withWriteLock(runnable); + }, + ioExecutor); + } + + private void withWriteLock(ThrowingRunnable runnable) throws IOException { + readWriteLock.writeLock().lock(); + try { + runnable.run(); + } finally { + readWriteLock.writeLock().unlock(); + } + } + + private CompletableFuture withReadLockAsync( + SupplierWithException runnable) { + return FutureUtils.supplyAsync(() -> withReadLock(runnable), ioExecutor); + } + + private T withReadLock(SupplierWithException supplier) throws IOException { + readWriteLock.readLock().lock(); + try { + return supplier.get(); + } finally { + readWriteLock.readLock().unlock(); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ApplicationResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ApplicationResult.java new file mode 100644 index 0000000000000..273cddf5f6f8f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ApplicationResult.java @@ -0,0 +1,151 @@ +/* + * 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.ApplicationID; +import org.apache.flink.api.common.ApplicationState; +import org.apache.flink.runtime.application.ArchivedApplication; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; + +/** + * The result of an application execution. This class collects information about a globally + * terminated application. + */ +public class ApplicationResult implements Serializable { + + private static final long serialVersionUID = 1L; + + private final ApplicationID applicationId; + private final ApplicationState applicationState; + private final String applicationName; + private final long startTime; + private final long endTime; + + private ApplicationResult( + ApplicationID applicationId, + ApplicationState applicationState, + String applicationName, + long startTime, + long endTime) { + this.applicationId = Preconditions.checkNotNull(applicationId); + this.applicationState = Preconditions.checkNotNull(applicationState); + this.applicationName = Preconditions.checkNotNull(applicationName); + this.startTime = startTime; + this.endTime = endTime; + } + + public ApplicationID getApplicationId() { + return applicationId; + } + + public ApplicationState getApplicationState() { + return applicationState; + } + + public String getApplicationName() { + return applicationName; + } + + public long getStartTime() { + return startTime; + } + + public long getEndTime() { + return endTime; + } + + /** + * Creates the {@link ApplicationResult} from the given {@link ArchivedApplication} which must + * be in a globally terminal state. + * + * @param archivedApplication to create the ApplicationResult from + * @return ApplicationResult of the given ArchivedApplication + */ + public static ApplicationResult createFrom(ArchivedApplication archivedApplication) { + final ApplicationID applicationId = archivedApplication.getApplicationId(); + final ApplicationState applicationState = archivedApplication.getApplicationStatus(); + + Preconditions.checkArgument( + applicationState.isTerminalState(), + "The application " + + archivedApplication.getApplicationName() + + '(' + + applicationId + + ") is not in a terminal state. It is in state " + + applicationState + + '.'); + + final ApplicationResult.Builder builder = new ApplicationResult.Builder(); + builder.applicationId(applicationId); + builder.applicationState(applicationState); + builder.applicationName(archivedApplication.getApplicationName()); + + final long startTime = archivedApplication.getStatusTimestamp(ApplicationState.CREATED); + final long endTime = archivedApplication.getStatusTimestamp(applicationState); + builder.startTime(startTime).endTime(endTime); + + return builder.build(); + } + + /** Builder for {@link ApplicationResult}. */ + public static class Builder { + + private ApplicationID applicationId; + + private ApplicationState applicationState; + + private String applicationName = "unknown"; + + private long startTime = -1; + + private long endTime = -1; + + public Builder applicationId(final ApplicationID applicationId) { + this.applicationId = applicationId; + return this; + } + + public Builder applicationState(final ApplicationState applicationState) { + this.applicationState = applicationState; + return this; + } + + public Builder applicationName(final String applicationName) { + this.applicationName = applicationName; + return this; + } + + public Builder startTime(final long startTime) { + this.startTime = startTime; + return this; + } + + public Builder endTime(final long endTime) { + this.endTime = endTime; + return this; + } + + public ApplicationResult build() { + return new ApplicationResult( + applicationId, applicationState, applicationName, startTime, endTime); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ApplicationResultEntry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ApplicationResultEntry.java new file mode 100644 index 0000000000000..ed0db1562f214 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ApplicationResultEntry.java @@ -0,0 +1,43 @@ +/* + * 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.ApplicationID; +import org.apache.flink.util.Preconditions; + +/** + * {@code ApplicationEntry} is the entity managed by the {@link ApplicationResultStore}. It collects + * information about a globally terminated application (e.g. {@link ApplicationResult}). + */ +public class ApplicationResultEntry { + + private final ApplicationResult applicationResult; + + public ApplicationResultEntry(ApplicationResult applicationResult) { + this.applicationResult = Preconditions.checkNotNull(applicationResult); + } + + public ApplicationResult getApplicationResult() { + return applicationResult; + } + + public ApplicationID getApplicationId() { + return applicationResult.getApplicationId(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ApplicationResultStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ApplicationResultStore.java new file mode 100644 index 0000000000000..5606e8e8169e9 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ApplicationResultStore.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.runtime.highavailability; + +import org.apache.flink.api.common.ApplicationID; + +import java.io.IOException; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +/** + * A storage for the results of globally terminated applications. These results can have the + * following states: + * + *
    + *
  • {@code dirty} - indicating that the corresponding application is not properly cleaned up, + * yet. + *
  • {@code clean} - indicating that the cleanup of the corresponding application is performed + * and no further actions need to be applied. + *
+ */ +public interface ApplicationResultStore { + + /** + * Registers the passed {@link ApplicationResultEntry} instance as {@code dirty} which indicates + * that clean-up operations still need to be performed. Once the application resource cleanup + * has been finalized, we can mark the {@code ApplicationEntry} as {@code clean} result using + * {@link #markResultAsCleanAsync(ApplicationID)}. + * + * @param applicationResultEntry The application result we wish to persist. + * @return a successfully completed future if the dirty result is created successfully. The + * future will be completed with {@link IllegalStateException} if the passed {@code + * applicationEntry} has an {@code ApplicationID} attached that is already registered in + * this {@code ApplicationResultStore}. + */ + CompletableFuture createDirtyResultAsync(ApplicationResultEntry applicationResultEntry); + + /** + * Marks an existing {@link ApplicationResultEntry} as {@code clean}. This indicates that no + * more resource cleanup steps need to be performed. No actions should be triggered if the + * passed {@code ApplicationID} belongs to an application that was already marked as clean. + * + * @param applicationId Ident of the application we wish to mark as clean. + * @return a successfully completed future if the result is marked successfully. The future can + * complete exceptionally with a {@link NoSuchElementException}. i.e. there is no + * corresponding {@code dirty} application present in the store for the given {@code + * ApplicationID}. + */ + CompletableFuture markResultAsCleanAsync(ApplicationID applicationId); + + /** + * Returns the future of whether the store already contains an entry for an application. + * + * @param applicationId Ident of the application we wish to check the store for. + * @return a successfully completed future with {@code true} if a {@code dirty} or {@code clean} + * {@link ApplicationResultEntry} exists for the given {@code ApplicationID}; otherwise + * {@code false}. + */ + default CompletableFuture hasApplicationResultEntryAsync(ApplicationID applicationId) { + return hasDirtyApplicationResultEntryAsync(applicationId) + .thenCombine( + hasCleanApplicationResultEntryAsync(applicationId), + (result1, result2) -> result1 || result2); + } + + /** + * Returns the future of whether the store contains a {@code dirty} entry for the given {@code + * ApplicationID}. + * + * @param applicationId Ident of the application we wish to check the store for. + * @return a successfully completed future with {@code true}, if a {@code dirty} entry exists + * for the given {@code ApplicationID}; otherwise {@code false}. + */ + CompletableFuture hasDirtyApplicationResultEntryAsync(ApplicationID applicationId); + + /** + * Returns the future of whether the store contains a {@code clean} entry for the given {@code + * ApplicationID}. + * + * @param applicationId Ident of the application we wish to check the store for. + * @return a successfully completed future with {@code true}, if a {@code clean} entry exists + * for the given {@code ApplicationID}; otherwise a successfully completed future with + * {@code false}. + */ + CompletableFuture hasCleanApplicationResultEntryAsync(ApplicationID applicationId); + + /** + * Get the persisted {@link ApplicationResult} instances that are marked as {@code dirty}. This + * is useful for recovery of finalization steps. + * + * @return A set of dirty {@code ApplicationResults} 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/ApplicationResultStoreOptions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ApplicationResultStoreOptions.java new file mode 100644 index 0000000000000..d992fd6bc5f05 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/ApplicationResultStoreOptions.java @@ -0,0 +1,71 @@ +/* + * 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.PublicEvolving; +import org.apache.flink.annotation.docs.Documentation; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.configuration.description.Description; +import org.apache.flink.configuration.description.TextElement; + +/** The set of configuration options relating to the Application Result Store. */ +@PublicEvolving +public class ApplicationResultStoreOptions { + + @Documentation.Section(Documentation.Sections.COMMON_HIGH_AVAILABILITY_APPLICATION_RESULT_STORE) + public static final ConfigOption STORAGE_PATH = + ConfigOptions.key("application-result-store.storage-path") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "Defines where application results should be stored. This should be an " + + "underlying file-system that provides read-after-write consistency. By " + + "default, this is %s.", + TextElement.code( + FileSystemApplicationResultStore + .createDefaultApplicationResultStorePath( + String.format( + "{%s}", + HighAvailabilityOptions + .HA_STORAGE_PATH + .key()), + String.format( + "{%s}", + HighAvailabilityOptions + .HA_CLUSTER_ID + .key())))) + .build()); + + @Documentation.Section(Documentation.Sections.COMMON_HIGH_AVAILABILITY_APPLICATION_RESULT_STORE) + public static final ConfigOption DELETE_ON_COMMIT = + ConfigOptions.key("application-result-store.delete-on-commit") + .booleanType() + .defaultValue(Boolean.TRUE) + .withDescription( + "Determines whether application results should be automatically removed " + + "from the underlying application result store when the corresponding entity " + + "transitions into a clean state. If false, the cleaned application results " + + "are, instead, marked as clean to indicate their state. In this " + + "case, Flink no longer has ownership and the resources need to " + + "be cleaned up by the user."); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedApplicationResultStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedApplicationResultStore.java new file mode 100644 index 0000000000000..af987d6dba3aa --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/EmbeddedApplicationResultStore.java @@ -0,0 +1,93 @@ +/* + * 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.ApplicationID; +import org.apache.flink.util.concurrent.Executors; + +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +/** An embedded implementation of {@link ApplicationResultStore} for testing purposes. */ +public class EmbeddedApplicationResultStore extends AbstractThreadsafeApplicationResultStore { + + private final Map dirtyResults = + new ConcurrentHashMap<>(); + private final Map cleanResults = + new ConcurrentHashMap<>(); + + public EmbeddedApplicationResultStore() { + super(Executors.directExecutor()); + } + + @Override + protected void createDirtyResultInternal(ApplicationResultEntry applicationResultEntry) { + dirtyResults.put(applicationResultEntry.getApplicationId(), applicationResultEntry); + } + + @Override + protected void markResultAsCleanInternal(ApplicationID applicationId) + throws NoSuchElementException { + final ApplicationResultEntry entry = dirtyResults.remove(applicationId); + if (entry != null) { + cleanResults.put(applicationId, entry); + } else { + throw new NoSuchElementException( + String.format( + "Could not mark application %s as clean as it is not present in the application result store.", + applicationId)); + } + } + + @Override + protected boolean hasDirtyApplicationResultEntryInternal(ApplicationID applicationId) { + return dirtyResults.containsKey(applicationId); + } + + @Override + protected boolean hasCleanApplicationResultEntryInternal(ApplicationID applicationId) { + return cleanResults.containsKey(applicationId); + } + + @Override + protected Set getDirtyResultsInternal() { + return dirtyResults.values().stream() + .map(ApplicationResultEntry::getApplicationResult) + .collect(Collectors.toSet()); + } + + /** Clears all stored results. */ + public void clear() { + dirtyResults.clear(); + cleanResults.clear(); + } + + /** Gets the number of dirty results. */ + public int getDirtyResultCount() { + return dirtyResults.size(); + } + + /** Gets the number of clean results. */ + public int getCleanResultCount() { + return cleanResults.size(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemApplicationResultStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemApplicationResultStore.java new file mode 100644 index 0000000000000..fd938fede9b15 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemApplicationResultStore.java @@ -0,0 +1,269 @@ +/* + * 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.VisibleForTesting; +import org.apache.flink.api.common.ApplicationID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.core.fs.FileStatus; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.fs.Path; +import org.apache.flink.runtime.rest.messages.json.ApplicationResultDeserializer; +import org.apache.flink.runtime.rest.messages.json.ApplicationResultSerializer; +import org.apache.flink.runtime.util.NonClosingOutputStreamDecorator; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.jackson.JacksonMapperFactory; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonSerialize; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.HashSet; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.Executor; + +import static org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly; + +/** + * An implementation of the {@link ApplicationResultStore} which persists application result data to + * an underlying distributed filesystem. + */ +public class FileSystemApplicationResultStore extends AbstractThreadsafeApplicationResultStore { + + private static final Logger LOG = + LoggerFactory.getLogger(FileSystemApplicationResultStore.class); + + @VisibleForTesting static final String FILE_EXTENSION = ".json"; + @VisibleForTesting static final String DIRTY_FILE_EXTENSION = "_DIRTY" + FILE_EXTENSION; + + @VisibleForTesting + public static boolean hasValidDirtyApplicationResultStoreEntryExtension(String filename) { + return filename.endsWith(DIRTY_FILE_EXTENSION); + } + + @VisibleForTesting + public static boolean hasValidApplicationResultStoreEntryExtension(String filename) { + return filename.endsWith(FILE_EXTENSION); + } + + private final ObjectMapper mapper = JacksonMapperFactory.createObjectMapper(); + + private final FileSystem fileSystem; + + private volatile boolean basePathCreated; + + private final Path basePath; + + private final boolean deleteOnCommit; + + @VisibleForTesting + FileSystemApplicationResultStore( + FileSystem fileSystem, Path basePath, boolean deleteOnCommit, Executor ioExecutor) { + super(ioExecutor); + this.fileSystem = fileSystem; + this.basePath = basePath; + this.deleteOnCommit = deleteOnCommit; + } + + public static FileSystemApplicationResultStore fromConfiguration( + Configuration config, Executor ioExecutor) throws IOException { + Preconditions.checkNotNull(config); + final String arsStoragePath = config.get(ApplicationResultStoreOptions.STORAGE_PATH); + final Path basePath; + + if (isNullOrWhitespaceOnly(arsStoragePath)) { + final String haStoragePath = config.get(HighAvailabilityOptions.HA_STORAGE_PATH); + final String haClusterId = config.get(HighAvailabilityOptions.HA_CLUSTER_ID); + basePath = + new Path(createDefaultApplicationResultStorePath(haStoragePath, haClusterId)); + } else { + basePath = new Path(arsStoragePath); + } + + boolean deleteOnCommit = config.get(ApplicationResultStoreOptions.DELETE_ON_COMMIT); + + return new FileSystemApplicationResultStore( + basePath.getFileSystem(), basePath, deleteOnCommit, ioExecutor); + } + + private void createBasePathIfNeeded() throws IOException { + if (!basePathCreated) { + LOG.info( + "Creating highly available application result storage directory at {}", + basePath); + fileSystem.mkdirs(basePath); + LOG.info( + "Created highly available application result storage directory at {}", + basePath); + basePathCreated = true; + } + } + + public static String createDefaultApplicationResultStorePath(String baseDir, String clusterId) { + return baseDir + "/application-result-store/" + clusterId; + } + + /** + * Given an application ID, construct the path for a dirty entry corresponding to it in the + * application result store. + * + * @param applicationId The application ID to construct a dirty entry path from. + * @return A path for a dirty entry for the given the Application ID. + */ + private Path constructDirtyPath(ApplicationID applicationId) { + return constructEntryPath(applicationId.toString() + DIRTY_FILE_EXTENSION); + } + + /** + * Given an application ID, construct the path for a clean entry corresponding to it in the + * application result store. + * + * @param applicationId The application ID to construct a clean entry path from. + * @return A path for a clean entry for the given the Application ID. + */ + private Path constructCleanPath(ApplicationID applicationId) { + return constructEntryPath(applicationId.toString() + FILE_EXTENSION); + } + + @VisibleForTesting + Path constructEntryPath(String fileName) { + return new Path(this.basePath, fileName); + } + + @Override + public void createDirtyResultInternal(ApplicationResultEntry applicationResultEntry) + throws IOException { + createBasePathIfNeeded(); + + final Path path = constructDirtyPath(applicationResultEntry.getApplicationId()); + try (OutputStream os = fileSystem.create(path, FileSystem.WriteMode.NO_OVERWRITE)) { + mapper.writeValue( + // working around the internally used _writeAndClose method to ensure that close + // is only called once + new NonClosingOutputStreamDecorator(os), + new JsonApplicationResultEntry(applicationResultEntry.getApplicationResult())); + } + } + + @Override + public void markResultAsCleanInternal(ApplicationID applicationId) + throws IOException, NoSuchElementException { + Path dirtyPath = constructDirtyPath(applicationId); + + if (!fileSystem.exists(dirtyPath)) { + throw new NoSuchElementException( + String.format( + "Could not mark application %s as clean as it is not present in the application result store.", + applicationId)); + } + + if (deleteOnCommit) { + fileSystem.delete(dirtyPath, false); + } else { + fileSystem.rename(dirtyPath, constructCleanPath(applicationId)); + } + } + + @Override + public boolean hasDirtyApplicationResultEntryInternal(ApplicationID applicationId) + throws IOException { + return fileSystem.exists(constructDirtyPath(applicationId)); + } + + @Override + public boolean hasCleanApplicationResultEntryInternal(ApplicationID applicationId) + throws IOException { + return fileSystem.exists(constructCleanPath(applicationId)); + } + + @Override + public Set getDirtyResultsInternal() throws IOException { + createBasePathIfNeeded(); + + final FileStatus[] statuses = fileSystem.listStatus(this.basePath); + + Preconditions.checkState( + statuses != null, + "The base directory of the ApplicationResultStore isn't accessible. No dirty ApplicationResults can be restored."); + + final Set dirtyResults = new HashSet<>(); + for (FileStatus s : statuses) { + if (!s.isDir()) { + if (hasValidDirtyApplicationResultStoreEntryExtension(s.getPath().getName())) { + JsonApplicationResultEntry jre = + mapper.readValue( + fileSystem.open(s.getPath()), JsonApplicationResultEntry.class); + ApplicationResult applicationResult = jre.getApplicationResult(); + if (applicationResult != null) { + dirtyResults.add(applicationResult); + } + } + } + } + return dirtyResults; + } + + /** + * Wrapper class around {@link ApplicationResultEntry} to allow for serialization of a schema + * version, so that future schema changes can be handled in a backwards compatible manner. + */ + @JsonIgnoreProperties( + value = {JsonApplicationResultEntry.FIELD_NAME_VERSION}, + allowGetters = true) + @VisibleForTesting + static class JsonApplicationResultEntry extends ApplicationResultEntry { + private static final String FIELD_NAME_RESULT = "result"; + static final String FIELD_NAME_VERSION = "version"; + + @JsonCreator + private JsonApplicationResultEntry( + @JsonProperty(FIELD_NAME_RESULT) ApplicationResult applicationResult) { + super(applicationResult); + } + + @Override + @JsonProperty(FIELD_NAME_RESULT) + @JsonSerialize(using = ApplicationResultSerializer.class) + @JsonDeserialize(using = ApplicationResultDeserializer.class) + public ApplicationResult getApplicationResult() { + return super.getApplicationResult(); + } + + @JsonIgnore + @Override + public ApplicationID getApplicationId() { + return super.getApplicationId(); + } + + public int getVersion() { + return 1; + } + } +} 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 971416e78b939..17c47a9221bbf 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 @@ -22,6 +22,7 @@ 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.jobmanager.ApplicationStore; import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; import org.apache.flink.runtime.leaderelection.LeaderElection; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -168,6 +169,22 @@ default LeaderElection getWebMonitorLeaderElection() { */ JobResultStore getJobResultStore() throws Exception; + /** + * Gets the submitted application store for the job manager. + * + * @return Submitted application store + * @throws Exception if the submitted application store could not be created + */ + ApplicationStore getApplicationStore() throws Exception; + + /** + * Gets the store that holds information about the state of finished applications. + * + * @return Store of finished application results + * @throws Exception if application result store could not be created + */ + ApplicationResultStore getApplicationResultStore() throws Exception; + /** * Creates the BLOB store in which BLOBs are stored in a highly-available fashion. * diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStoreOptions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStoreOptions.java index c8341df1116a5..c9493b0abc636 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStoreOptions.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/JobResultStoreOptions.java @@ -55,7 +55,11 @@ public class JobResultStoreOptions { .key())))) .build()); - @Documentation.Section(Documentation.Sections.COMMON_HIGH_AVAILABILITY_JOB_RESULT_STORE) + /** + * @deprecated Use {@link ApplicationResultStoreOptions#DELETE_ON_COMMIT} + */ + @Deprecated + @Documentation.ExcludeFromDocumentation("Hidden for deprecated") public static final ConfigOption DELETE_ON_COMMIT = ConfigOptions.key("job-result-store.delete-on-commit") .booleanType() 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 5af64336fc4e1..e0eb2344462f9 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 @@ -22,10 +22,14 @@ 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.ApplicationResultStore; +import org.apache.flink.runtime.highavailability.EmbeddedApplicationResultStore; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; +import org.apache.flink.runtime.jobmanager.ApplicationStore; import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; +import org.apache.flink.runtime.jobmanager.StandaloneApplicationStore; import org.apache.flink.runtime.jobmanager.StandaloneExecutionPlanStore; import javax.annotation.concurrent.GuardedBy; @@ -45,12 +49,18 @@ public abstract class AbstractNonHaServices implements HighAvailabilityServices private final JobResultStore jobResultStore; + private final ApplicationStore applicationStore; + + private final ApplicationResultStore applicationResultStore; + private final VoidBlobStore voidBlobStore; private boolean shutdown; public AbstractNonHaServices() { this.jobResultStore = new EmbeddedJobResultStore(); + this.applicationStore = new StandaloneApplicationStore(); + this.applicationResultStore = new EmbeddedApplicationResultStore(); this.voidBlobStore = new VoidBlobStore(); shutdown = false; @@ -87,6 +97,24 @@ public JobResultStore getJobResultStore() throws Exception { } } + @Override + public ApplicationStore getApplicationStore() throws Exception { + synchronized (lock) { + checkNotShutdown(); + + return applicationStore; + } + } + + @Override + public ApplicationResultStore getApplicationResultStore() throws Exception { + synchronized (lock) { + checkNotShutdown(); + + return applicationResultStore; + } + } + @Override public BlobStore createBlobStore() throws IOException { synchronized (lock) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperLeaderElectionHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperLeaderElectionHaServices.java index e866034e485b8..bd1c78109b9b7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperLeaderElectionHaServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/zookeeper/ZooKeeperLeaderElectionHaServices.java @@ -24,7 +24,9 @@ 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.FileSystemApplicationResultStore; import org.apache.flink.runtime.highavailability.FileSystemJobResultStore; +import org.apache.flink.runtime.jobmanager.ApplicationStore; import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; import org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionDriverFactory; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -77,7 +79,8 @@ public ZooKeeperLeaderElectionHaServices( ZooKeeperUtils.getLeaderPath())), executor, blobStoreService, - FileSystemJobResultStore.fromConfiguration(configuration, executor)); + FileSystemJobResultStore.fromConfiguration(configuration, executor), + FileSystemApplicationResultStore.fromConfiguration(configuration, executor)); this.curatorFrameworkWrapper = checkNotNull(curatorFrameworkWrapper); } @@ -96,6 +99,12 @@ public ExecutionPlanStore createExecutionPlanStore() throws Exception { curatorFrameworkWrapper.asCuratorFramework(), configuration); } + @Override + public ApplicationStore createApplicationStore() throws Exception { + return ZooKeeperUtils.createApplicationStore( + curatorFrameworkWrapper.asCuratorFramework(), configuration); + } + @Override protected void internalClose() { curatorFrameworkWrapper.close(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ApplicationStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ApplicationStore.java new file mode 100644 index 0000000000000..88a6cd8e3260c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ApplicationStore.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.jobmanager; + +import org.apache.flink.api.common.ApplicationID; + +import java.util.Collection; +import java.util.Optional; + +/** {@link ApplicationStoreEntry} instances for recovery. */ +public interface ApplicationStore extends ApplicationWriter { + + /** Starts the {@link ApplicationStore} service. */ + void start() throws Exception; + + /** Stops the {@link ApplicationStore} service. */ + void stop() throws Exception; + + /** + * Returns the {@link ApplicationStoreEntry} with the given {@link ApplicationID} or {@link + * Optional#empty()} if no application was registered. + */ + Optional recoverApplication(ApplicationID applicationId) + throws Exception; + + /** + * Get all application ids of submitted applications to the submitted application store. + * + * @return Collection of submitted application ids + * @throws Exception if the operation fails + */ + Collection getApplicationIds() throws Exception; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ApplicationStoreEntry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ApplicationStoreEntry.java new file mode 100644 index 0000000000000..10a55473122c5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ApplicationStoreEntry.java @@ -0,0 +1,58 @@ +/* + * 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.api.common.ApplicationID; +import org.apache.flink.api.common.JobInfo; +import org.apache.flink.runtime.application.AbstractApplication; +import org.apache.flink.runtime.blob.PermanentBlobService; + +import java.io.Serializable; +import java.util.Collection; + +/** Entry of {@code ApplicationStore}. */ +public interface ApplicationStoreEntry extends Serializable { + + /** + * Get the application from this entry. + * + * @param blobService The blob service to retrieve user artifacts + * @param recoveredJobInfos The info of jobs recovered from a previous run + * @param recoveredTerminalJobInfos The info of terminal jobs from a previous run + * @return The application + */ + AbstractApplication getApplication( + PermanentBlobService blobService, + Collection recoveredJobInfos, + Collection recoveredTerminalJobInfos); + + /** + * Gets the unique identifier of the application. + * + * @return the application id + */ + ApplicationID getApplicationId(); + + /** + * Gets the name of the application. + * + * @return the application name + */ + String getName(); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ApplicationStoreUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ApplicationStoreUtil.java new file mode 100644 index 0000000000000..0ec9f9d47c51c --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ApplicationStoreUtil.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.jobmanager; + +import org.apache.flink.api.common.ApplicationID; + +/** + * ApplicationStore utility interfaces. For example, convert a name(e.g. ZooKeeper path, key name in + * Kubernetes ConfigMap) to {@link ApplicationID}, or vice versa. + */ +public interface ApplicationStoreUtil { + + /** + * Get the name in external storage from application id. + * + * @param applicationId application id + * @return Key name in ConfigMap or child path name in ZooKeeper + */ + String applicationIdToName(ApplicationID applicationId); + + /** + * Get the application id from name. + * + * @param name Key name in ConfigMap or child path name in ZooKeeper + * @return parsed application id. + */ + ApplicationID nameToApplicationId(String name); +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ApplicationWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ApplicationWriter.java new file mode 100644 index 0000000000000..f49956b412e63 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ApplicationWriter.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.jobmanager; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ApplicationID; +import org.apache.flink.runtime.dispatcher.cleanup.GloballyCleanableApplicationResource; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; + +/** Allows to store and remove applications. */ +@Internal +public interface ApplicationWriter extends GloballyCleanableApplicationResource { + + /** + * Adds the {@link ApplicationStoreEntry} instance. + * + *

If an application with the same {@link ApplicationID} exists, it is replaced. + */ + void putApplication(ApplicationStoreEntry application) throws Exception; + + @Override + default CompletableFuture globalCleanupAsync( + ApplicationID applicationId, Executor executor) { + return CompletableFuture.completedFuture(null); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultApplicationStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultApplicationStore.java new file mode 100644 index 0000000000000..fc47b69bbad14 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultApplicationStore.java @@ -0,0 +1,328 @@ +/* + * 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.api.common.ApplicationID; +import org.apache.flink.runtime.persistence.ResourceVersion; +import org.apache.flink.runtime.persistence.StateHandleStore; +import org.apache.flink.runtime.state.RetrievableStateHandle; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.function.ThrowingRunnable; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.GuardedBy; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.Executor; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Default implementation for {@link ApplicationStore}. Combined with different {@link + * StateHandleStore}, we could persist the applications to various distributed storage. + */ +public class DefaultApplicationStore> implements ApplicationStore { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultApplicationStore.class); + + private final Object lock = new Object(); + + /** The set of IDs of all added applications. */ + @GuardedBy("lock") + private final Set addedApplications = new HashSet<>(); + + /** Submitted applications handle store. */ + private final StateHandleStore applicationStateHandleStore; + + private final ApplicationStoreUtil applicationStoreUtil; + + /** Flag indicating whether this instance is running. */ + @GuardedBy("lock") + private volatile boolean running; + + public DefaultApplicationStore( + StateHandleStore stateHandleStore, + ApplicationStoreUtil applicationStoreUtil) { + this.applicationStateHandleStore = checkNotNull(stateHandleStore); + this.applicationStoreUtil = checkNotNull(applicationStoreUtil); + + this.running = false; + } + + @Override + public void start() throws Exception { + synchronized (lock) { + if (!running) { + running = true; + } + } + } + + @Override + public void stop() throws Exception { + synchronized (lock) { + if (running) { + running = false; + LOG.info("Stopping DefaultApplicationStore."); + Exception exception = null; + + try { + applicationStateHandleStore.releaseAll(); + } catch (Exception e) { + exception = e; + } + + if (exception != null) { + throw new FlinkException( + "Could not properly stop the DefaultApplicationStore.", exception); + } + } + } + } + + @Override + public Optional recoverApplication(ApplicationID applicationId) + throws Exception { + checkNotNull(applicationId, "Application ID"); + + LOG.debug("Recovering application {} from {}.", applicationId, applicationStateHandleStore); + + final String name = applicationStoreUtil.applicationIdToName(applicationId); + + synchronized (lock) { + verifyIsRunning(); + + boolean success = false; + + RetrievableStateHandle applicationRetrievableStateHandle; + + try { + try { + applicationRetrievableStateHandle = + applicationStateHandleStore.getAndLock(name); + } catch (StateHandleStore.NotExistException ignored) { + success = true; + return Optional.empty(); + } catch (Exception e) { + throw new FlinkException( + "Could not retrieve the submitted application state handle " + + "for " + + name + + " from the submitted application store.", + e); + } + + ApplicationStoreEntry application; + try { + application = applicationRetrievableStateHandle.retrieveState(); + } catch (ClassNotFoundException cnfe) { + throw new FlinkException( + "Could not retrieve submitted application from state handle under " + + name + + ". This indicates that you are trying to recover from state written by an " + + "older Flink version which is not compatible. Try cleaning the state handle store.", + cnfe); + } catch (IOException ioe) { + throw new FlinkException( + "Could not retrieve submitted application from state handle under " + + name + + ". This indicates that the retrieved state handle is broken. Try cleaning the state handle " + + "store.", + ioe); + } + + addedApplications.add(applicationId); + + LOG.info("Recovered {} ({}).", application.getName(), applicationId); + + success = true; + return Optional.of(application); + } finally { + if (!success) { + applicationStateHandleStore.release(name); + } + } + } + } + + @Override + public void putApplication(ApplicationStoreEntry application) throws Exception { + checkNotNull(application, "Application"); + + final ApplicationID applicationID = application.getApplicationId(); + final String name = applicationStoreUtil.applicationIdToName(applicationID); + + LOG.debug("Adding application {} to {}.", applicationID, applicationStateHandleStore); + + boolean success = false; + + while (!success) { + synchronized (lock) { + verifyIsRunning(); + + final R currentVersion = applicationStateHandleStore.exists(name); + + if (!currentVersion.isExisting()) { + try { + applicationStateHandleStore.addAndLock(name, application); + + addedApplications.add(applicationID); + + success = true; + } catch (StateHandleStore.AlreadyExistException ignored) { + LOG.warn( + "{} already exists in {}.", + application, + applicationStateHandleStore); + } + } else if (addedApplications.contains(applicationID)) { + try { + applicationStateHandleStore.replace(name, currentVersion, application); + LOG.info("Updated {} in {}.", application, getClass().getSimpleName()); + + success = true; + } catch (StateHandleStore.NotExistException ignored) { + LOG.warn( + "{} does not exists in {}.", + application, + applicationStateHandleStore); + } + } else { + throw new IllegalStateException( + "Trying to update an application you didn't " + + "#getAllSubmittedApplications() or #putApplication() yourself before."); + } + } + } + + LOG.info("Added {} to {}.", application, applicationStateHandleStore); + } + + @Override + public CompletableFuture globalCleanupAsync( + ApplicationID applicationId, Executor executor) { + checkNotNull(applicationId, "Application ID"); + + return runAsyncWithLockAssertRunning( + () -> { + LOG.debug( + "Removing application {} from {}.", + applicationId, + applicationStateHandleStore); + + final String name = applicationStoreUtil.applicationIdToName(applicationId); + releaseAndRemoveOrThrowCompletionException(applicationId, name); + + addedApplications.remove(applicationId); + + LOG.info( + "Removed application {} from {}.", + applicationId, + applicationStateHandleStore); + }, + executor); + } + + @GuardedBy("lock") + private void releaseAndRemoveOrThrowCompletionException( + ApplicationID applicationId, String applicationName) { + boolean success; + try { + success = applicationStateHandleStore.releaseAndTryRemove(applicationName); + } catch (Exception e) { + throw new CompletionException(e); + } + + if (!success) { + throw new CompletionException( + new FlinkException( + String.format( + "Could not remove application with application id %s from %s.", + applicationId, applicationStateHandleStore))); + } + } + + private CompletableFuture runAsyncWithLockAssertRunning( + ThrowingRunnable runnable, Executor executor) { + return CompletableFuture.runAsync( + () -> { + synchronized (lock) { + verifyIsRunning(); + try { + runnable.run(); + } catch (Exception e) { + throw new CompletionException(e); + } + } + }, + executor); + } + + @Override + public Collection getApplicationIds() throws Exception { + LOG.debug("Retrieving all stored application ids from {}.", applicationStateHandleStore); + + final Collection names; + try { + names = applicationStateHandleStore.getAllHandles(); + } catch (Exception e) { + throw new Exception( + "Failed to retrieve all application ids from " + + applicationStateHandleStore + + ".", + e); + } + + final List applicationIds = new ArrayList<>(names.size()); + + for (String name : names) { + try { + applicationIds.add(applicationStoreUtil.nameToApplicationId(name)); + } catch (Exception exception) { + LOG.warn( + "Could not parse application id from {}. This indicates a malformed name.", + name, + exception); + } + } + + LOG.info( + "Retrieved application ids {} from {}", + applicationIds, + applicationStateHandleStore); + + return applicationIds; + } + + /** Verifies that the state is running. */ + private void verifyIsRunning() { + checkState(running, "Not running. Forgot to call start()?"); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobPersistenceComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesPersistenceComponentFactory.java similarity index 76% rename from flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobPersistenceComponentFactory.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesPersistenceComponentFactory.java index 064184da682ee..44ca012e4557f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesJobPersistenceComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/HaServicesPersistenceComponentFactory.java @@ -18,19 +18,20 @@ package org.apache.flink.runtime.jobmanager; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; 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 JobPersistenceComponentFactory} implementation which creates a {@link ExecutionPlanStore} + * {@link PersistenceComponentFactory} implementation which creates a {@link ExecutionPlanStore} * using the provided {@link HighAvailabilityServices}. */ -public class HaServicesJobPersistenceComponentFactory implements JobPersistenceComponentFactory { +public class HaServicesPersistenceComponentFactory implements PersistenceComponentFactory { private final HighAvailabilityServices highAvailabilityServices; - public HaServicesJobPersistenceComponentFactory( + public HaServicesPersistenceComponentFactory( HighAvailabilityServices highAvailabilityServices) { this.highAvailabilityServices = highAvailabilityServices; } @@ -45,6 +46,17 @@ public JobResultStore createJobResultStore() { return create(highAvailabilityServices::getJobResultStore, JobResultStore.class); } + @Override + public ApplicationStore createApplicationStore() { + return create(highAvailabilityServices::getApplicationStore, ApplicationStore.class); + } + + @Override + public ApplicationResultStore createApplicationResultStore() { + return create( + highAvailabilityServices::getApplicationResultStore, ApplicationResultStore.class); + } + private T create(SupplierWithException supplier, Class clazz) { try { return supplier.get(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobPersistenceComponentFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/PersistenceComponentFactory.java similarity index 73% rename from flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobPersistenceComponentFactory.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/PersistenceComponentFactory.java index 7834d7d39722b..256f972db4059 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobPersistenceComponentFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/PersistenceComponentFactory.java @@ -18,10 +18,11 @@ package org.apache.flink.runtime.jobmanager; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; import org.apache.flink.runtime.highavailability.JobResultStore; /** Factory for components that are responsible for persisting a job for recovery. */ -public interface JobPersistenceComponentFactory { +public interface PersistenceComponentFactory { /** * Creates a {@link ExecutionPlanStore}. @@ -36,4 +37,18 @@ public interface JobPersistenceComponentFactory { * @return a {@code JobResultStore} instance. */ JobResultStore createJobResultStore(); + + /** + * Creates {@link ApplicationStore} instances. + * + * @return a {@code ApplicationStore} instance. + */ + ApplicationStore createApplicationStore(); + + /** + * Creates {@link ApplicationResultStore} instances. + * + * @return a {@code ApplicationResultStore} instance. + */ + ApplicationResultStore createApplicationResultStore(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneApplicationStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneApplicationStore.java new file mode 100644 index 0000000000000..d35f3ed7e7ee4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/StandaloneApplicationStore.java @@ -0,0 +1,59 @@ +/* + * 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.api.common.ApplicationID; + +import java.util.Collection; +import java.util.Collections; +import java.util.Optional; + +/** + * {@link ApplicationStore} instances for JobManagers running in {@link HighAvailabilityMode#NONE}. + * + *

All operations are NoOps, because {@link ApplicationStoreEntry} instances cannot be recovered + * in this recovery mode. + */ +public class StandaloneApplicationStore implements ApplicationStore { + + @Override + public void start() throws Exception { + // Nothing to do + } + + @Override + public void stop() { + // Nothing to do + } + + @Override + public void putApplication(ApplicationStoreEntry application) { + // Nothing to do + } + + @Override + public Collection getApplicationIds() { + return Collections.emptyList(); + } + + @Override + public Optional recoverApplication(ApplicationID applicationID) { + return Optional.empty(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperApplicationStoreUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperApplicationStoreUtil.java new file mode 100644 index 0000000000000..3682ead2e9327 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/ZooKeeperApplicationStoreUtil.java @@ -0,0 +1,37 @@ +/* + * 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.api.common.ApplicationID; +import org.apache.flink.runtime.util.ZooKeeperUtils; + +/** Singleton {@link ApplicationStoreUtil} implementation for ZooKeeper. */ +public enum ZooKeeperApplicationStoreUtil implements ApplicationStoreUtil { + INSTANCE; + + @Override + public String applicationIdToName(ApplicationID applicationId) { + return ZooKeeperUtils.getPathForApplication(applicationId); + } + + @Override + public ApplicationID nameToApplicationId(String name) { + return ApplicationID.fromHexString(name); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java index 7d2defb29fb82..0b834ca503277 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/minicluster/MiniCluster.java @@ -20,6 +20,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ApplicationID; +import org.apache.flink.api.common.ApplicationState; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; @@ -35,6 +37,7 @@ import org.apache.flink.core.execution.RecoveryClaimMode; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.runtime.application.AbstractApplication; +import org.apache.flink.runtime.application.ArchivedApplication; import org.apache.flink.runtime.application.SingleJobApplication; import org.apache.flink.runtime.blob.BlobCacheService; import org.apache.flink.runtime.blob.BlobClient; @@ -1182,6 +1185,18 @@ private CompletableFuture createBlobServerAddress( .thenCompose(Function.identity()); } + // ------------------------------------------------------------------------ + // Accessing applications + // ------------------------------------------------------------------------ + + public CompletableFuture getApplicationStatus(ApplicationID applicationId) { + return runDispatcherCommand( + dispatcherGateway -> + dispatcherGateway + .requestApplication(applicationId, rpcTimeout) + .thenApply(ArchivedApplication::getApplicationStatus)); + } + // ------------------------------------------------------------------------ // factories - can be overridden by subclasses to alter behavior // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/ApplicationResultDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/ApplicationResultDeserializer.java new file mode 100644 index 0000000000000..223a83b54c88f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/ApplicationResultDeserializer.java @@ -0,0 +1,128 @@ +/* + * 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.rest.messages.json; + +import org.apache.flink.api.common.ApplicationID; +import org.apache.flink.api.common.ApplicationState; +import org.apache.flink.runtime.highavailability.ApplicationResult; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonToken; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonMappingException; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdDeserializer; + +import javax.annotation.Nullable; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkState; + +/** + * JSON deserializer for {@link ApplicationResult}. + * + * @see ApplicationResultSerializer + */ +public class ApplicationResultDeserializer extends StdDeserializer { + + private static final long serialVersionUID = 1L; + + private final ApplicationIDDeserializer applicationIdDeserializer = + new ApplicationIDDeserializer(); + + private final SerializedThrowableDeserializer serializedThrowableDeserializer = + new SerializedThrowableDeserializer(); + + public ApplicationResultDeserializer() { + super(ApplicationResult.class); + } + + @Override + public ApplicationResult deserialize(final JsonParser p, final DeserializationContext ctxt) + throws IOException { + ApplicationID applicationId = null; + ApplicationState applicationState = null; + String applicationName = "unknown"; + long startTime = -1; + long endTime = -1; + + while (true) { + final JsonToken jsonToken = p.nextToken(); + assertNotEndOfInput(p, jsonToken); + if (jsonToken == JsonToken.END_OBJECT) { + break; + } + + final String fieldName = p.getValueAsString(); + switch (fieldName) { + case ApplicationResultSerializer.FIELD_NAME_APPLICATION_ID: + assertNextToken(p, JsonToken.VALUE_STRING); + applicationId = applicationIdDeserializer.deserialize(p, ctxt); + break; + case ApplicationResultSerializer.FIELD_NAME_APPLICATION_STATE: + assertNextToken(p, JsonToken.VALUE_STRING); + applicationState = ApplicationState.valueOf(p.getValueAsString()); + break; + case ApplicationResultSerializer.FIELD_NAME_APPLICATION_NAME: + assertNextToken(p, JsonToken.VALUE_STRING); + applicationName = p.getValueAsString(); + break; + case ApplicationResultSerializer.FIELD_NAME_START_TIME: + assertNextToken(p, JsonToken.VALUE_NUMBER_INT); + startTime = p.getLongValue(); + break; + case ApplicationResultSerializer.FIELD_NAME_END_TIME: + assertNextToken(p, JsonToken.VALUE_NUMBER_INT); + endTime = p.getLongValue(); + break; + default: + // ignore unknown fields + } + } + + try { + return new ApplicationResult.Builder() + .applicationId(applicationId) + .applicationState(applicationState) + .applicationName(applicationName) + .startTime(startTime) + .endTime(endTime) + .build(); + } catch (final RuntimeException e) { + throw new JsonMappingException( + null, "Could not deserialize " + ApplicationResult.class.getSimpleName(), e); + } + } + + /** Asserts that the provided JsonToken is not null, i.e., not at the end of the input. */ + private static void assertNotEndOfInput( + final JsonParser p, @Nullable final JsonToken jsonToken) { + checkState(jsonToken != null, "Unexpected end of input at %s", p.getCurrentLocation()); + } + + /** Advances the token and asserts that it matches the required {@link JsonToken}. */ + private static void assertNextToken(final JsonParser p, final JsonToken requiredJsonToken) + throws IOException { + final JsonToken jsonToken = p.nextToken(); + if (jsonToken != requiredJsonToken) { + throw new JsonMappingException( + p, String.format("Expected token %s (was %s)", requiredJsonToken, jsonToken)); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/ApplicationResultSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/ApplicationResultSerializer.java new file mode 100644 index 0000000000000..dc73a18f4913a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/ApplicationResultSerializer.java @@ -0,0 +1,77 @@ +/* + * 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.rest.messages.json; + +import org.apache.flink.runtime.highavailability.ApplicationResult; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.StdSerializer; + +import java.io.IOException; + +/** + * JSON serializer for {@link ApplicationResult}. + * + * @see ApplicationResultDeserializer + */ +public class ApplicationResultSerializer extends StdSerializer { + + private static final long serialVersionUID = 1L; + + static final String FIELD_NAME_APPLICATION_ID = "application-id"; + + static final String FIELD_NAME_APPLICATION_STATE = "application-state"; + + static final String FIELD_NAME_APPLICATION_NAME = "application-name"; + + static final String FIELD_NAME_START_TIME = "start-time"; + + static final String FIELD_NAME_END_TIME = "end-time"; + + private final ApplicationIDSerializer applicationIdSerializer = new ApplicationIDSerializer(); + + public ApplicationResultSerializer() { + super(ApplicationResult.class); + } + + @Override + public void serialize( + final ApplicationResult result, + final JsonGenerator gen, + final SerializerProvider provider) + throws IOException { + + gen.writeStartObject(); + + gen.writeFieldName(FIELD_NAME_APPLICATION_ID); + applicationIdSerializer.serialize(result.getApplicationId(), gen, provider); + + gen.writeFieldName(FIELD_NAME_APPLICATION_STATE); + gen.writeString(result.getApplicationState().name()); + + gen.writeFieldName(FIELD_NAME_APPLICATION_NAME); + gen.writeString(result.getApplicationName()); + + gen.writeNumberField(FIELD_NAME_START_TIME, result.getStartTime()); + gen.writeNumberField(FIELD_NAME_END_TIME, result.getEndTime()); + + gen.writeEndObject(); + } +} 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 a1ed8f78f0e3e..e814c35987efa 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 @@ -19,6 +19,7 @@ package org.apache.flink.runtime.util; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.ApplicationID; import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; @@ -35,9 +36,13 @@ import org.apache.flink.runtime.checkpoint.ZooKeeperCheckpointStoreUtil; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener; +import org.apache.flink.runtime.jobmanager.ApplicationStore; +import org.apache.flink.runtime.jobmanager.ApplicationStoreEntry; +import org.apache.flink.runtime.jobmanager.DefaultApplicationStore; import org.apache.flink.runtime.jobmanager.DefaultExecutionPlanStore; import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; +import org.apache.flink.runtime.jobmanager.ZooKeeperApplicationStoreUtil; import org.apache.flink.runtime.jobmanager.ZooKeeperExecutionPlanStoreUtil; import org.apache.flink.runtime.jobmanager.ZooKeeperExecutionPlanStoreWatcher; import org.apache.flink.runtime.leaderelection.LeaderInformation; @@ -102,6 +107,9 @@ public class ZooKeeperUtils { public static final String HA_STORAGE_SUBMITTED_EXECUTION_PLAN_PREFIX = "submittedExecutionPlan"; + /** The prefix of the submitted application file. */ + public static final String HA_STORAGE_SUBMITTED_APPLICATION_PREFIX = "submittedApplication"; + /** The prefix of the completed checkpoint file. */ public static final String HA_STORAGE_COMPLETED_CHECKPOINT = "completedCheckpoint"; @@ -561,6 +569,44 @@ public static ExecutionPlanStore createExecutionPlans( ZooKeeperExecutionPlanStoreUtil.INSTANCE); } + /** + * Creates a {@link DefaultApplicationStore} instance with {@link ZooKeeperStateHandleStore}, + * and {@link ZooKeeperApplicationStoreUtil}. + * + * @param client The {@link CuratorFramework} ZooKeeper client to use + * @param configuration {@link Configuration} object + * @return {@link DefaultApplicationStore} instance + * @throws Exception if the submitted application store cannot be created + */ + public static ApplicationStore createApplicationStore( + CuratorFramework client, Configuration configuration) throws Exception { + + checkNotNull(configuration, "Configuration"); + + RetrievableStateStorageHelper stateStorage = + createFileSystemStateStorage( + configuration, HA_STORAGE_SUBMITTED_APPLICATION_PREFIX); + + // ZooKeeper submitted applications root dir + String zooKeeperApplicationsPath = + configuration.get(HighAvailabilityOptions.HA_ZOOKEEPER_APPLICATIONS_PATH); + + // Ensure that the applications path exists + client.newNamespaceAwareEnsurePath(zooKeeperApplicationsPath) + .ensure(client.getZookeeperClient()); + + // All operations will have the path as root + CuratorFramework facade = + client.usingNamespace(client.getNamespace() + zooKeeperApplicationsPath); + + final ZooKeeperStateHandleStore + zooKeeperApplicationStateHandleStore = + new ZooKeeperStateHandleStore<>(facade, stateStorage); + + return new DefaultApplicationStore<>( + zooKeeperApplicationStateHandleStore, ZooKeeperApplicationStoreUtil.INSTANCE); + } + /** * Creates a {@link DefaultCompletedCheckpointStore} instance with {@link * ZooKeeperStateHandleStore}. @@ -616,6 +662,12 @@ public static String getPathForJob(JobID jobId) { return String.format("/%s", jobId); } + /** Returns the ApplicationID as a String (with leading slash). */ + public static String getPathForApplication(ApplicationID applicationId) { + checkNotNull(applicationId, "Application ID"); + return String.format("/%s", applicationId); + } + /** * Creates an instance of {@link ZooKeeperStateHandleStore}. * 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 b50a1826367a0..8c16371c29a4d 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 @@ -26,8 +26,10 @@ import org.apache.flink.runtime.dispatcher.cleanup.CheckpointResourcesCleanupRunnerFactory; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.heartbeat.HeartbeatServicesImpl; +import org.apache.flink.runtime.highavailability.EmbeddedApplicationResultStore; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; +import org.apache.flink.runtime.jobmanager.StandaloneApplicationStore; import org.apache.flink.runtime.jobmanager.StandaloneExecutionPlanStore; import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; import org.apache.flink.runtime.rpc.RpcUtils; @@ -97,6 +99,8 @@ public void setUp() throws Exception { haServices.setResourceManagerLeaderRetriever(new SettableLeaderRetrievalService()); haServices.setExecutionPlanStore(new StandaloneExecutionPlanStore()); haServices.setJobResultStore(new EmbeddedJobResultStore()); + haServices.setApplicationStore(new StandaloneApplicationStore()); + haServices.setApplicationResultStore(new EmbeddedApplicationResultStore()); configuration = new Configuration(); blobServer = @@ -110,6 +114,8 @@ protected TestingDispatcher.Builder createTestingDispatcherBuilder() { .setHighAvailabilityServices(haServices) .setExecutionPlanWriter(haServices.getExecutionPlanStore()) .setJobResultStore(haServices.getJobResultStore()) + .setApplicationWriter(haServices.getApplicationStore()) + .setApplicationResultStore(haServices.getApplicationResultStore()) .setJobManagerRunnerFactory(JobMasterServiceLeadershipRunnerFactory.INSTANCE) .setCleanupRunnerFactory(CheckpointResourcesCleanupRunnerFactory.INSTANCE) .setFatalErrorHandler(testingFatalErrorHandlerResource.getFatalErrorHandler()) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherApplicationResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherApplicationResourceCleanupTest.java new file mode 100644 index 0000000000000..269593e26e809 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherApplicationResourceCleanupTest.java @@ -0,0 +1,383 @@ +/* + * 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.ApplicationID; +import org.apache.flink.api.common.ApplicationState; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.BlobUtils; +import org.apache.flink.runtime.blob.TestingBlobStoreBuilder; +import org.apache.flink.runtime.dispatcher.cleanup.TestingApplicationResourceCleanerFactory; +import org.apache.flink.runtime.highavailability.ApplicationResultEntry; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.testutils.TestingApplicationResultStore; +import org.apache.flink.runtime.util.TestingFatalErrorHandlerExtension; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Reference; +import org.apache.flink.util.TestLoggerExtension; +import org.apache.flink.util.concurrent.FutureUtils; + +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 org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.nio.file.Path; +import java.time.Duration; +import java.util.concurrent.CancellationException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.fail; + +/** Tests the application resource cleanup by the {@link Dispatcher}. */ +@ExtendWith(TestLoggerExtension.class) +public class DispatcherApplicationResourceCleanupTest { + + @TempDir public Path tempDir; + + @RegisterExtension + final TestingFatalErrorHandlerExtension testingFatalErrorHandlerResource = + new TestingFatalErrorHandlerExtension(); + + private static final Duration timeout = Duration.ofSeconds(10L); + + private static TestingRpcService rpcService; + + private ApplicationID applicationId; + + private TestingDispatcher dispatcher; + + private DispatcherGateway dispatcherGateway; + + private BlobServer blobServer; + + private CompletableFuture globalCleanupFuture; + + @BeforeAll + public static void setupClass() { + rpcService = new TestingRpcService(); + } + + @BeforeEach + void setup() throws Exception { + applicationId = new ApplicationID(); + + globalCleanupFuture = new CompletableFuture<>(); + + blobServer = + BlobUtils.createBlobServer( + new Configuration(), + Reference.owned(tempDir.toFile()), + new TestingBlobStoreBuilder().createTestingBlobStore()); + } + + @AfterEach + void teardown() throws Exception { + if (dispatcher != null) { + dispatcher.close(); + } + + if (blobServer != null) { + blobServer.close(); + } + } + + @AfterAll + static void teardownClass() throws ExecutionException, InterruptedException { + if (rpcService != null) { + rpcService.closeAsync().get(); + } + } + + @Test + void testGlobalCleanupWhenApplicationFinished() throws Exception { + startDispatcher(createTestingDispatcherBuilder()); + + submitApplicationAndWait(); + + dispatcher.notifyApplicationStatusChange(applicationId, ApplicationState.FINISHED); + + dispatcher + .getApplicationTerminationFuture(applicationId) + .get(timeout.toMillis(), TimeUnit.MILLISECONDS); + + assertGlobalCleanupTriggered(applicationId); + } + + @Test + void testApplicationMarkedAsDirtyBeforeCleanup() throws Exception { + final OneShotLatch markAsDirtyLatch = new OneShotLatch(); + + final TestingDispatcher.Builder dispatcherBuilder = + createTestingDispatcherBuilder() + .setApplicationResultStore( + TestingApplicationResultStore.builder() + .withCreateDirtyResultConsumer( + ignoredApplicationResultEntry -> { + try { + markAsDirtyLatch.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return FutureUtils.completedExceptionally( + e); + } + return FutureUtils.completedVoidFuture(); + }) + .build()); + startDispatcher(dispatcherBuilder); + + submitApplicationAndWait(); + + dispatcher.notifyApplicationStatusChange(applicationId, ApplicationState.FINISHED); + + assertThatNoCleanupWasTriggered(); + + // Trigger the latch to allow dirty result creation to complete + markAsDirtyLatch.trigger(); + + // Now cleanup should be triggered + assertGlobalCleanupTriggered(applicationId); + } + + @Test + void testApplicationMarkedAsCleanAfterCleanup() throws Exception { + final CompletableFuture markAsCleanFuture = new CompletableFuture<>(); + + final ApplicationResultStore applicationResultStore = + TestingApplicationResultStore.builder() + .withMarkResultAsCleanConsumer( + applicationId -> { + markAsCleanFuture.complete(applicationId); + return FutureUtils.completedVoidFuture(); + }) + .build(); + + final OneShotLatch cleanupLatch = new OneShotLatch(); + + final TestingApplicationResourceCleanerFactory resourceCleanerFactory = + TestingApplicationResourceCleanerFactory.builder() + .withGloballyCleanableResource( + (applicationId, ignoredExecutor) -> { + try { + cleanupLatch.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + + return FutureUtils.completedVoidFuture(); + }) + .build(); + + final TestingDispatcher.Builder dispatcherBuilder = + createTestingDispatcherBuilder() + .setApplicationResultStore(applicationResultStore) + .setApplicationResourceCleanerFactory(resourceCleanerFactory); + + startDispatcher(dispatcherBuilder); + + submitApplicationAndWait(); + + dispatcher.notifyApplicationStatusChange(applicationId, ApplicationState.FINISHED); + + // Mark as clean should not have been called yet + assertFalse(markAsCleanFuture.isDone()); + + // Trigger cleanup + cleanupLatch.trigger(); + + // Wait for cleanup to complete + dispatcher + .getApplicationTerminationFuture(applicationId) + .get(timeout.toMillis(), TimeUnit.MILLISECONDS); + + // Verify mark as clean was called + assertEquals( + applicationId, markAsCleanFuture.get(timeout.toMillis(), TimeUnit.MILLISECONDS)); + } + + @Test + void testDispatcherTerminationTerminatesRunningApplications() throws Exception { + startDispatcher(createTestingDispatcherBuilder()); + + submitApplicationAndWait(); + + dispatcher.closeAsync().get(); + + assertThrows( + CancellationException.class, + () -> dispatcher.getApplicationTerminationFuture(applicationId).get()); + } + + @Test + void testFatalErrorIfApplicationCannotBeMarkedDirtyInApplicationResultStore() throws Exception { + final ApplicationResultStore applicationResultStore = + TestingApplicationResultStore.builder() + .withCreateDirtyResultConsumer( + applicationResult -> + FutureUtils.completedExceptionally( + new IOException("Expected IOException."))) + .build(); + + startDispatcher( + createTestingDispatcherBuilder().setApplicationResultStore(applicationResultStore)); + + submitApplicationAndWait(); + + dispatcher.notifyApplicationStatusChange(applicationId, ApplicationState.FINISHED); + + // Fatal error should be reported + final CompletableFuture errorFuture = + testingFatalErrorHandlerResource.getTestingFatalErrorHandler().getErrorFuture(); + assertThat(errorFuture.get()).isInstanceOf(FlinkException.class); + + testingFatalErrorHandlerResource.getTestingFatalErrorHandler().clearError(); + } + + @Test + void testErrorHandlingIfApplicationCannotBeMarkedAsCleanInApplicationResultStore() + throws Exception { + final CompletableFuture dirtyApplicationFuture = + new CompletableFuture<>(); + final ApplicationResultStore applicationResultStore = + TestingApplicationResultStore.builder() + .withCreateDirtyResultConsumer( + applicationResultEntry -> { + dirtyApplicationFuture.complete(applicationResultEntry); + return FutureUtils.completedVoidFuture(); + }) + .withMarkResultAsCleanConsumer( + applicationId -> + FutureUtils.completedExceptionally( + new IOException("Expected IOException."))) + .build(); + + startDispatcher( + createTestingDispatcherBuilder().setApplicationResultStore(applicationResultStore)); + + submitApplicationAndWait(); + + dispatcher.notifyApplicationStatusChange(applicationId, ApplicationState.FINISHED); + + // No fatal error should be reported (mark as clean failure is handled gracefully) + final CompletableFuture errorFuture = + testingFatalErrorHandlerResource.getTestingFatalErrorHandler().getErrorFuture(); + try { + errorFuture.get(100, TimeUnit.MILLISECONDS); + fail("No error should have been reported."); + } catch (TimeoutException e) { + // expected + } + + // Dirty result should have been created + assertEquals( + applicationId, + dirtyApplicationFuture + .get(timeout.toMillis(), TimeUnit.MILLISECONDS) + .getApplicationId()); + } + + @Test + void testArchivingFinishedApplicationToHistoryServer() throws Exception { + final CompletableFuture archiveFuture = new CompletableFuture<>(); + + final TestingDispatcher.Builder dispatcherBuilder = + createTestingDispatcherBuilder() + .setHistoryServerArchivist( + TestingHistoryServerArchivist.builder() + .setArchiveApplicationFunction( + archivedApplication -> archiveFuture) + .build()); + + startDispatcher(dispatcherBuilder); + + submitApplicationAndWait(); + + dispatcher.notifyApplicationStatusChange(applicationId, ApplicationState.FINISHED); + + // Before the archiving is finished, the cleanup is not finished and the application is not + // terminated + assertThatNoCleanupWasTriggered(); + assertFalse(dispatcher.getApplicationTerminationFuture(applicationId).isDone()); + + // Complete archiving + archiveFuture.complete(Acknowledge.get()); + + // Once the archive is finished, the cleanup is finished and the application is terminated. + assertGlobalCleanupTriggered(applicationId); + dispatcher.getApplicationTerminationFuture(applicationId).join(); + } + + private void startDispatcher(TestingDispatcher.Builder dispatcherBuilder) throws Exception { + dispatcher = dispatcherBuilder.build(rpcService); + + dispatcher.start(); + + dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); + } + + private TestingDispatcher.Builder createTestingDispatcherBuilder() { + return TestingDispatcher.builder() + .setBlobServer(blobServer) + .setFatalErrorHandler( + testingFatalErrorHandlerResource.getTestingFatalErrorHandler()) + .setApplicationResourceCleanerFactory( + TestingApplicationResourceCleanerFactory.builder() + .withGloballyCleanableResource( + (applicationId, ignoredExecutor) -> { + globalCleanupFuture.complete(applicationId); + return FutureUtils.completedVoidFuture(); + }) + .build()); + } + + private void assertThatNoCleanupWasTriggered() { + assertThat(globalCleanupFuture.isDone()).isFalse(); + } + + private void assertGlobalCleanupTriggered(ApplicationID applicationId) + throws ExecutionException, InterruptedException, TimeoutException { + assertThat(globalCleanupFuture.get(timeout.toMillis(), TimeUnit.MILLISECONDS)) + .isEqualTo(applicationId); + } + + private CompletableFuture submitApplication() { + return dispatcherGateway.submitApplication( + TestingApplication.builder().setApplicationId(applicationId).build(), timeout); + } + + private void submitApplicationAndWait() { + submitApplication().join(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherApplicationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherApplicationTest.java new file mode 100644 index 0000000000000..15748868ffd4c --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherApplicationTest.java @@ -0,0 +1,848 @@ +/* + * 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.ApplicationID; +import org.apache.flink.api.common.ApplicationState; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.testutils.FlinkAssertions; +import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.application.AbstractApplication; +import org.apache.flink.runtime.application.ArchivedApplication; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.PermanentBlobKey; +import org.apache.flink.runtime.blob.VoidBlobStore; +import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; +import org.apache.flink.runtime.client.DuplicateApplicationSubmissionException; +import org.apache.flink.runtime.client.JobSubmissionException; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.dispatcher.cleanup.CheckpointResourcesCleanupRunnerFactory; +import org.apache.flink.runtime.dispatcher.cleanup.TestingCleanupRunnerFactory; +import org.apache.flink.runtime.highavailability.ApplicationResult; +import org.apache.flink.runtime.highavailability.ApplicationResultEntry; +import org.apache.flink.runtime.highavailability.EmbeddedApplicationResultStore; +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.jobgraph.JobGraphTestUtils; +import org.apache.flink.runtime.jobmanager.StandaloneApplicationStore; +import org.apache.flink.runtime.jobmanager.StandaloneExecutionPlanStore; +import org.apache.flink.runtime.jobmaster.JobResult; +import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; +import org.apache.flink.runtime.leaderretrieval.SettableLeaderRetrievalService; +import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.messages.FlinkApplicationTerminatedWithoutCancellationException; +import org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder; +import org.apache.flink.runtime.rpc.TestingRpcService; +import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; +import org.apache.flink.runtime.testutils.TestingApplicationResultStore; +import org.apache.flink.runtime.util.TestingFatalErrorHandlerExtension; +import org.apache.flink.util.TestLoggerExtension; + +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 org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static org.apache.flink.core.testutils.FlinkAssertions.assertThatFuture; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Tests for the {@link Dispatcher} component that are related to applications. */ +@ExtendWith(TestLoggerExtension.class) +public class DispatcherApplicationTest { + + static TestingRpcService rpcService; + + static final Duration TIMEOUT = Duration.ofMinutes(1L); + + @TempDir public Path tempDir; + + @RegisterExtension + final TestingFatalErrorHandlerExtension testingFatalErrorHandlerResource = + new TestingFatalErrorHandlerExtension(); + + Configuration configuration; + + BlobServer blobServer; + + TestingHighAvailabilityServices haServices; + + private ApplicationID applicationId; + + private JobGraph jobGraph; + + private JobID jobId; + + /** Instance under test. */ + private TestingDispatcher dispatcher; + + @BeforeAll + public static void setupClass() { + rpcService = new TestingRpcService(); + } + + @BeforeEach + void beforeEach() throws Exception { + haServices = new TestingHighAvailabilityServices(); + haServices.setCheckpointRecoveryFactory(new StandaloneCheckpointRecoveryFactory()); + haServices.setResourceManagerLeaderRetriever(new SettableLeaderRetrievalService()); + haServices.setExecutionPlanStore(new StandaloneExecutionPlanStore()); + haServices.setJobResultStore(new EmbeddedJobResultStore()); + haServices.setApplicationStore(new StandaloneApplicationStore()); + haServices.setApplicationResultStore(new EmbeddedApplicationResultStore()); + + configuration = new Configuration(); + blobServer = new BlobServer(configuration, tempDir.toFile(), new VoidBlobStore()); + jobGraph = JobGraphTestUtils.singleNoOpJobGraph(); + jobId = jobGraph.getJobID(); + applicationId = ApplicationID.fromHexString(jobId.toHexString()); + jobGraph.setApplicationId(applicationId); + } + + @AfterEach + void teardown() throws Exception { + if (dispatcher != null) { + dispatcher.close(); + } + + if (blobServer != null) { + blobServer.close(); + } + } + + @AfterAll + static void teardownClass() throws ExecutionException, InterruptedException { + if (rpcService != null) { + rpcService.closeAsync().get(); + } + } + + @Test + void testApplicationStatusChange_ArchiveNotCalledForNonTerminalStatus() throws Exception { + final CompletableFuture archiveApplicationFuture = new CompletableFuture<>(); + dispatcher = + createTestingDispatcherBuilder() + .setHistoryServerArchivist( + TestingHistoryServerArchivist.builder() + .setArchiveApplicationFunction( + archivedApplication -> { + archiveApplicationFuture.complete(null); + return CompletableFuture.completedFuture(null); + }) + .build()) + .build(rpcService); + dispatcher.start(); + submitApplicationAndMockApplicationStatusChange(ApplicationState.RUNNING); + + // verify that archive application is not called + assertFalse(archiveApplicationFuture.isDone()); + assertFalse(dispatcher.getApplicationTerminationFuture(applicationId).isDone()); + } + + @Test + void testApplicationStatusChange_ArchiveCalledForTerminalStatus() throws Exception { + final CompletableFuture archiveApplicationFuture = new CompletableFuture<>(); + dispatcher = + createTestingDispatcherBuilder() + .setHistoryServerArchivist( + TestingHistoryServerArchivist.builder() + .setArchiveApplicationFunction( + archivedApplication -> { + archiveApplicationFuture.complete( + archivedApplication.getApplicationId()); + return CompletableFuture.completedFuture(null); + }) + .build()) + .build(rpcService); + dispatcher.start(); + submitApplicationAndMockApplicationStatusChange(ApplicationState.FINISHED); + + // verify that archive application is called with the application id + assertEquals(applicationId, archiveApplicationFuture.get()); + dispatcher + .getApplicationTerminationFuture(applicationId) + .get(TIMEOUT.toMillis(), TimeUnit.MILLISECONDS); + } + + @Test + void testApplicationStatusChange_ThrowsIfDuplicateTerminalStatus() throws Exception { + dispatcher = createTestingDispatcherBuilder().build(rpcService); + dispatcher.start(); + submitApplicationAndMockApplicationStatusChange(ApplicationState.FINISHED); + // wait for archive to complete + dispatcher + .getApplicationTerminationFuture(applicationId) + .get(TIMEOUT.toMillis(), TimeUnit.MILLISECONDS); + + assertThrows( + IllegalStateException.class, + () -> + dispatcher.notifyApplicationStatusChange( + applicationId, ApplicationState.FAILED)); + } + + @Test + void testApplicationBootstrap() throws Exception { + final OneShotLatch bootstrapLatch = new OneShotLatch(); + final AbstractApplication application = + TestingApplication.builder() + .setApplicationId(applicationId) + .setExecuteFunction( + ignored -> { + bootstrapLatch.trigger(); + return CompletableFuture.completedFuture(Acknowledge.get()); + }) + .build(); + + dispatcher = + createTestingDispatcherBuilder() + .setDispatcherBootstrapFactory( + (ignoredDispatcherGateway, + ignoredScheduledExecutor, + ignoredFatalErrorHandler) -> + new ApplicationBootstrap(application)) + .setJobManagerRunnerFactory( + new TestingJobMasterServiceLeadershipRunnerFactory()) + .build(rpcService); + + dispatcher.start(); + + // ensure that the application execution is triggered + bootstrapLatch.await(); + + assertThat(dispatcher.getApplications().size()).isEqualTo(1); + assertThat(dispatcher.getApplications().keySet()).contains(applicationId); + + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); + + assertThat(application.getJobs().size()).isEqualTo(1); + assertThat(application.getJobs()).contains(jobId); + } + + @Test + public void testApplicationSubmission() throws Exception { + dispatcher = createTestingDispatcherBuilder().build(rpcService); + dispatcher.start(); + DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); + + final CompletableFuture submittedApplicationFuture = + new CompletableFuture<>(); + final AbstractApplication application = + TestingApplication.builder() + .setApplicationId(applicationId) + .setExecuteFunction( + ignored -> { + submittedApplicationFuture.complete(applicationId); + return CompletableFuture.completedFuture(Acknowledge.get()); + }) + .build(); + + dispatcherGateway.submitApplication(application, TIMEOUT).get(); + + // ensure that the application execution is triggered + assertThat(submittedApplicationFuture).isCompletedWithValue(applicationId); + + ArchivedApplication archivedApplication = + dispatcher.requestApplication(applicationId, TIMEOUT).get(); + assertThat(archivedApplication.getApplicationId()).isEqualTo(applicationId); + } + + @Test + public void testDuplicateApplicationSubmission() throws Exception { + dispatcher = createTestingDispatcherBuilder().build(rpcService); + dispatcher.start(); + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + + final AbstractApplication application = + TestingApplication.builder().setApplicationId(applicationId).build(); + // submit application + dispatcherGateway.submitApplication(application, TIMEOUT).get(); + + // duplicate submission + final CompletableFuture submitFuture = + dispatcherGateway.submitApplication(application, TIMEOUT); + assertThatThrownBy(submitFuture::get) + .hasCauseInstanceOf(DuplicateApplicationSubmissionException.class); + } + + @Test + public void testDuplicateApplicationSubmissionIsDetectedOnSimultaneousSubmission() + throws Exception { + dispatcher = createTestingDispatcherBuilder().build(rpcService); + dispatcher.start(); + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + + final AbstractApplication application = + TestingApplication.builder().setApplicationId(applicationId).build(); + + final int numThreads = 5; + final CountDownLatch prepareLatch = new CountDownLatch(numThreads); + final OneShotLatch startLatch = new OneShotLatch(); + + final Collection exceptions = Collections.synchronizedList(new ArrayList<>()); + final Collection threads = new ArrayList<>(); + for (int x = 0; x < numThreads; x++) { + threads.add( + new Thread( + () -> { + try { + prepareLatch.countDown(); + startLatch.awaitQuietly(); + dispatcherGateway + .submitApplication(application, TIMEOUT) + .join(); + } catch (Throwable t) { + exceptions.add(t); + } + })); + } + + // start worker threads and trigger submissions + threads.forEach(Thread::start); + prepareLatch.await(); + startLatch.trigger(); + + // wait for the submissions to happen + for (Thread thread : threads) { + thread.join(); + } + + // verify the application was actually submitted + ArchivedApplication archivedApplication = + dispatcher.requestApplication(applicationId, TIMEOUT).get(); + assertThat(archivedApplication.getApplicationId()).isEqualTo(applicationId); + + // verify that all but one submission failed as duplicates + assertThat(exceptions) + .hasSize(numThreads - 1) + .allSatisfy( + t -> + assertThat(t) + .hasCauseInstanceOf( + DuplicateApplicationSubmissionException.class)); + } + + @Test + public void testApplicationCancellation() throws Exception { + dispatcher = createTestingDispatcherBuilder().build(rpcService); + dispatcher.start(); + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + + final CompletableFuture canceledApplicationFuture = new CompletableFuture<>(); + final AbstractApplication application = + TestingApplication.builder() + .setApplicationId(applicationId) + .setCancelFunction( + ignored -> { + canceledApplicationFuture.complete(null); + return null; + }) + .build(); + + dispatcherGateway.submitApplication(application, TIMEOUT).get(); + + // verify the application was actually submitted + ArchivedApplication archivedApplication = + dispatcher.requestApplication(applicationId, TIMEOUT).get(); + assertThat(archivedApplication.getApplicationId()).isEqualTo(applicationId); + + // submission has succeeded, now cancel the application + dispatcherGateway.cancelApplication(applicationId, TIMEOUT).get(); + + assertThatFuture(canceledApplicationFuture).isDone(); + } + + @Test + public void testApplicationCancellationOfCanceledTerminalDoesNotThrowException() + throws Exception { + dispatcher = createTestingDispatcherBuilder().build(rpcService); + dispatcher.start(); + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + + final AbstractApplication application = + TestingApplication.builder() + .setApplicationId(applicationId) + .setGetApplicationStatusFunction(ignored -> ApplicationState.CANCELED) + .build(); + + dispatcherGateway.submitApplication(application, TIMEOUT).get(); + + // verify the application was actually submitted + ArchivedApplication archivedApplication = + dispatcher.requestApplication(applicationId, TIMEOUT).get(); + assertThat(archivedApplication.getApplicationId()).isEqualTo(applicationId); + assertThat(archivedApplication.getApplicationStatus()).isEqualTo(ApplicationState.CANCELED); + + // cancel the application should not throw + dispatcherGateway.cancelApplication(applicationId, TIMEOUT).get(); + } + + @Test + public void testApplicationCancellationOfNonCanceledTerminalFailsWithAppropriateException() + throws Exception { + dispatcher = createTestingDispatcherBuilder().build(rpcService); + dispatcher.start(); + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + + final AbstractApplication application = + TestingApplication.builder() + .setApplicationId(applicationId) + .setGetApplicationStatusFunction(ignored -> ApplicationState.FINISHED) + .build(); + + dispatcherGateway.submitApplication(application, TIMEOUT).get(); + + // verify the application was actually submitted + ArchivedApplication archivedApplication = + dispatcher.requestApplication(applicationId, TIMEOUT).get(); + assertThat(archivedApplication.getApplicationId()).isEqualTo(applicationId); + assertThat(archivedApplication.getApplicationStatus()).isEqualTo(ApplicationState.FINISHED); + + // cancel the application should throw + final CompletableFuture cancelFuture = + dispatcherGateway.cancelApplication(applicationId, TIMEOUT); + + FlinkAssertions.assertThatFuture(cancelFuture) + .eventuallyFails() + .withCauseOfType(FlinkApplicationTerminatedWithoutCancellationException.class); + } + + @Test + public void testShutDownFutureCompletesAfterApplicationArchivingFutures() throws Exception { + final CompletableFuture archiveApplicationFuture = new CompletableFuture<>(); + dispatcher = + createTestingDispatcherBuilder() + .setHistoryServerArchivist( + TestingHistoryServerArchivist.builder() + .setArchiveApplicationFunction( + archivedApplication -> archiveApplicationFuture) + .build()) + .build(rpcService); + dispatcher.start(); + + submitApplicationAndMockApplicationStatusChange(ApplicationState.FINISHED); + + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + + dispatcherGateway.shutDownCluster(ApplicationStatus.SUCCEEDED).get(); + assertThatThrownBy(() -> dispatcher.getShutDownFuture().get(100L, TimeUnit.MILLISECONDS)) + .isInstanceOf(TimeoutException.class); + + archiveApplicationFuture.complete(null); + + dispatcher.getShutDownFuture().get(); + } + + @Test + public void testRecoverJobSuccessfully() throws Exception { + final TestingJobMasterServiceLeadershipRunnerFactory jobManagerRunnerFactory = + new TestingJobMasterServiceLeadershipRunnerFactory(); + + dispatcher = + createTestingDispatcherBuilder() + .setJobManagerRunnerFactory(jobManagerRunnerFactory) + .setRecoveredJobs(Collections.singleton(jobGraph)) + .setDispatcherBootstrapFactory( + (ignoredDispatcherGateway, + ignoredScheduledExecutor, + ignoredFatalErrorHandler) -> + new ApplicationBootstrap( + TestingApplication.builder() + .setApplicationId(applicationId) + .build())) + .build(rpcService); + + dispatcher.start(); + dispatcher.waitUntilStarted(); + + // verify that the recovered job is NOT recovered immediately + assertThat(jobManagerRunnerFactory.getQueueSize()).isZero(); + assertThat(dispatcher.getSuspendedJobs().containsKey(jobId)).isTrue(); + assertThat(dispatcher.getSuspendedJobIdsByApplicationId().containsKey(applicationId)) + .isTrue(); + + // call recoverJob RPC + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + dispatcherGateway.recoverJob(jobId, TIMEOUT).get(); + + // verify that the job is now recovered + final TestingJobManagerRunner jobManagerRunner = + jobManagerRunnerFactory.takeCreatedJobManagerRunner(); + assertThat(jobManagerRunner.getJobID()).isEqualTo(jobId); + assertThat(dispatcher.getSuspendedJobs().containsKey(jobId)).isFalse(); + assertThat(dispatcher.getSuspendedJobIdsByApplicationId().containsKey(applicationId)) + .isFalse(); + } + + @Test + public void testRecoverJobFailsWhenJobNotFound() throws Exception { + dispatcher = + createTestingDispatcherBuilder() + .setRecoveredJobs(Collections.singleton(jobGraph)) + .setDispatcherBootstrapFactory( + (ignoredDispatcherGateway, + ignoredScheduledExecutor, + ignoredFatalErrorHandler) -> + new ApplicationBootstrap( + TestingApplication.builder() + .setApplicationId(applicationId) + .build())) + .build(rpcService); + + dispatcher.start(); + dispatcher.waitUntilStarted(); + + // try to recover a job that doesn't exist + final JobID unknownJobId = new JobID(); + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + final CompletableFuture recoverFuture = + dispatcherGateway.recoverJob(unknownJobId, TIMEOUT); + + assertThatThrownBy(recoverFuture::get) + .hasCauseInstanceOf(JobSubmissionException.class) + .hasMessageContaining("Cannot find the recovered job"); + } + + @Test + public void testRemainingSuspendedJobsCleanedWhenApplicationReachesTerminalState() + throws Exception { + final TestingJobMasterServiceLeadershipRunnerFactory jobManagerRunnerFactory = + new TestingJobMasterServiceLeadershipRunnerFactory(); + final TestingCleanupRunnerFactory cleanupRunnerFactory = new TestingCleanupRunnerFactory(); + + dispatcher = + createTestingDispatcherBuilder() + .setJobManagerRunnerFactory(jobManagerRunnerFactory) + .setCleanupRunnerFactory(cleanupRunnerFactory) + .setRecoveredJobs(Collections.singleton(jobGraph)) + .setDispatcherBootstrapFactory( + (ignoredDispatcherGateway, + ignoredScheduledExecutor, + ignoredFatalErrorHandler) -> + new ApplicationBootstrap( + TestingApplication.builder() + .setApplicationId(applicationId) + .build())) + .build(rpcService); + + dispatcher.start(); + dispatcher.waitUntilStarted(); + + // verify that the recovered job exists + assertThat(dispatcher.getSuspendedJobs().containsKey(jobId)).isTrue(); + assertThat(dispatcher.getSuspendedJobIdsByApplicationId().containsKey(applicationId)) + .isTrue(); + + // complete the application - this should trigger cleanup of the remaining recovered job + dispatcher + .callAsyncInMainThread( + () -> { + dispatcher.notifyApplicationStatusChange( + applicationId, ApplicationState.FINISHED); + return CompletableFuture.completedFuture(Acknowledge.get()); + }) + .get(); + + // verify that no jobs are recovered + assertThat(jobManagerRunnerFactory.getQueueSize()).isZero(); + + // verify that the remaining recovered job is cleaned up + final TestingJobManagerRunner cleanupRunner = + cleanupRunnerFactory.takeCreatedJobManagerRunner(); + assertThat(cleanupRunner.getJobID()).isEqualTo(jobId); + assertThat(dispatcher.getSuspendedJobs().containsKey(jobId)).isFalse(); + assertThat(dispatcher.getSuspendedJobIdsByApplicationId().containsKey(applicationId)) + .isFalse(); + } + + @Test + public void testJobResultNotMarkedCleanUntilApplicationTerminates() throws Exception { + final TestingJobMasterServiceLeadershipRunnerFactory jobManagerRunnerFactory = + new TestingJobMasterServiceLeadershipRunnerFactory(); + + dispatcher = + createTestingDispatcherBuilder() + .setJobManagerRunnerFactory(jobManagerRunnerFactory) + .setDispatcherBootstrapFactory( + (ignoredDispatcherGateway, + ignoredScheduledExecutor, + ignoredFatalErrorHandler) -> + new ApplicationBootstrap( + TestingApplication.builder() + .setApplicationId(applicationId) + .build())) + .build(rpcService); + + dispatcher.start(); + dispatcher.waitUntilStarted(); + + // submit a job + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); + + // complete the job + final TestingJobManagerRunner jobManagerRunner = + jobManagerRunnerFactory.takeCreatedJobManagerRunner(); + final ExecutionGraphInfo completedExecutionGraphInfo = + new ExecutionGraphInfo( + new ArchivedExecutionGraphBuilder() + .setJobID(jobId) + .setState(JobStatus.FINISHED) + .setApplicationId(applicationId) + .build()); + jobManagerRunner.completeResultFuture(completedExecutionGraphInfo); + + // job termination future should not be completed + assertThatThrownBy( + () -> + dispatcher + .getJobTerminationFuture(jobId, TIMEOUT) + .get(10L, TimeUnit.MILLISECONDS)) + .isInstanceOf(TimeoutException.class); + + // verify that the job result is NOT marked clean yet + assertThat( + haServices.getJobResultStore().getDirtyResults().stream() + .anyMatch(r -> r.getJobId().equals(jobId))) + .isTrue(); + + // complete the application + dispatcher.notifyApplicationStatusChange(applicationId, ApplicationState.FINISHED); + + // wait for application termination + dispatcher + .getApplicationTerminationFuture(applicationId) + .get(TIMEOUT.toMillis(), TimeUnit.MILLISECONDS); + + // wait for job termination + dispatcher + .getJobTerminationFuture(jobId, TIMEOUT) + .get(TIMEOUT.toMillis(), TimeUnit.MILLISECONDS); + + // verify that the job result is now marked clean + assertThat( + haServices.getJobResultStore().getDirtyResults().stream() + .noneMatch(r -> r.getJobId().equals(jobId))) + .isTrue(); + } + + @Test + public void testRecoveredDirtyJobResultsCleanedOnApplicationSubmission() throws Exception { + // create a dirty job result + final JobResult jobResult = + new JobResult.Builder() + .jobId(jobId) + .jobStatus(JobStatus.FINISHED) + .netRuntime(1) + .applicationId(applicationId) + .build(); + + final TestingCleanupRunnerFactory cleanupRunnerFactory = new TestingCleanupRunnerFactory(); + + dispatcher = + createTestingDispatcherBuilder() + .setCleanupRunnerFactory(cleanupRunnerFactory) + .setRecoveredDirtyJobs(Collections.singleton(jobResult)) + .setDispatcherBootstrapFactory( + (ignoredDispatcherGateway, + ignoredScheduledExecutor, + ignoredFatalErrorHandler) -> + new ApplicationBootstrap( + TestingApplication.builder() + .setApplicationId(applicationId) + .build())) + .build(rpcService); + + // verify that the dirty job result exists + assertThat( + dispatcher + .getRecoveredDirtyJobResultsByApplicationId() + .containsKey(applicationId)) + .isTrue(); + + // start application - this should trigger the cleanup of dirty job results + dispatcher.start(); + dispatcher.waitUntilStarted(); + + // verify that the dirty job result was cleaned up + final TestingJobManagerRunner cleanupRunner = + cleanupRunnerFactory.takeCreatedJobManagerRunner(); + assertThat(cleanupRunner.getJobID()).isEqualTo(jobId); + assertThat( + dispatcher + .getRecoveredDirtyJobResultsByApplicationId() + .containsKey(applicationId)) + .isFalse(); + } + + @Test + public void testDuplicateSubmissionWithRecoveredApplication() throws Exception { + final AbstractApplication application = + TestingApplication.builder().setApplicationId(applicationId).build(); + dispatcher = + createTestingDispatcherBuilder() + .setRecoveredApplications(Collections.singleton(application)) + .build(rpcService); + dispatcher.start(); + dispatcher.waitUntilStarted(); + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + final CompletableFuture submitFuture = + dispatcherGateway.submitApplication(application, TIMEOUT); + + assertThatThrownBy(submitFuture::get) + .hasCauseInstanceOf(DuplicateApplicationSubmissionException.class); + } + + @Test + public void testThatDirtilyFinishedApplicationsNotRetriggered() { + final AbstractApplication application = + TestingApplication.builder().setApplicationId(applicationId).build(); + final ApplicationResult applicationResult = + TestingApplicationResultStore.createSuccessfulApplicationResult(applicationId); + + assertThrows( + IllegalArgumentException.class, + () -> + createTestingDispatcherBuilder() + .setRecoveredApplications(Collections.singleton(application)) + .setRecoveredDirtyApplications( + Collections.singleton(applicationResult)) + .build(rpcService)); + } + + @Test + public void testApplicationCleanupWithoutRecoveredApplication() throws Exception { + final ApplicationResult applicationResult = + TestingApplicationResultStore.createSuccessfulApplicationResult(applicationId); + haServices + .getApplicationResultStore() + .createDirtyResultAsync(new ApplicationResultEntry(applicationResult)) + .get(); + dispatcher = + createTestingDispatcherBuilder() + .setRecoveredDirtyApplications(Collections.singleton(applicationResult)) + .build(rpcService); + + dispatcher.start(); + dispatcher.waitUntilStarted(); + + ArchivedApplication application = + dispatcher.requestApplication(applicationId, TIMEOUT).get(); + + assertThat(application.getApplicationId()).isEqualTo(applicationId); + assertThat(application.getApplicationStatus()).isEqualTo(ApplicationState.FINISHED); + + dispatcher.getApplicationTerminationFuture(applicationId).get(); + + assertTrue( + haServices + .getApplicationResultStore() + .hasCleanApplicationResultEntryAsync(applicationId) + .get()); + } + + @Test + public void testOnlyRecoveredApplicationsAreRetainedInTheBlobServer() throws Exception { + final ApplicationID applicationId1 = new ApplicationID(); + final ApplicationID applicationId2 = new ApplicationID(); + final byte[] fileContent = {1, 2, 3, 4}; + final PermanentBlobKey blobKey1 = blobServer.putPermanent(applicationId1, fileContent); + final PermanentBlobKey blobKey2 = blobServer.putPermanent(applicationId2, fileContent); + + final AbstractApplication application1 = + TestingApplication.builder().setApplicationId(applicationId1).build(); + + dispatcher = + createTestingDispatcherBuilder() + .setRecoveredApplications(Collections.singleton(application1)) + .build(rpcService); + + assertThat(blobServer.getFile(applicationId1, blobKey1)).hasBinaryContent(fileContent); + assertThatThrownBy(() -> blobServer.getFile(applicationId2, blobKey2)) + .isInstanceOf(NoSuchFileException.class); + } + + private void submitApplicationAndMockApplicationStatusChange(ApplicationState targetState) + throws Exception { + submitApplication(); + mockApplicationStatusChange(targetState); + } + + private void submitApplication() throws Exception { + dispatcher + .submitApplication( + TestingApplication.builder().setApplicationId(applicationId).build(), + TIMEOUT) + .get(); + } + + private void mockApplicationStatusChange(ApplicationState targetState) throws Exception { + dispatcher.notifyApplicationStatusChange(applicationId, targetState); + } + + private TestingDispatcher.Builder createTestingDispatcherBuilder() { + return TestingDispatcher.builder() + .setConfiguration(configuration) + .setHighAvailabilityServices(haServices) + .setExecutionPlanWriter(haServices.getExecutionPlanStore()) + .setJobResultStore(haServices.getJobResultStore()) + .setApplicationWriter(haServices.getApplicationStore()) + .setApplicationResultStore(haServices.getApplicationResultStore()) + .setJobManagerRunnerFactory(JobMasterServiceLeadershipRunnerFactory.INSTANCE) + .setCleanupRunnerFactory(CheckpointResourcesCleanupRunnerFactory.INSTANCE) + .setFatalErrorHandler( + testingFatalErrorHandlerResource.getTestingFatalErrorHandler()) + .setBlobServer(blobServer); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java index 0bc934334a80f..e9fc9833be580 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java @@ -17,18 +17,27 @@ package org.apache.flink.runtime.dispatcher; +import org.apache.flink.api.common.ApplicationID; +import org.apache.flink.api.common.ApplicationState; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.configuration.CleanupOptions; import org.apache.flink.core.execution.RecoveryClaimMode; import org.apache.flink.core.testutils.OneShotLatch; +import org.apache.flink.runtime.application.AbstractApplication; +import org.apache.flink.runtime.application.ArchivedApplication; import org.apache.flink.runtime.application.SingleJobApplication; import org.apache.flink.runtime.checkpoint.EmbeddedCompletedCheckpointStore; import org.apache.flink.runtime.checkpoint.PerJobCheckpointRecoveryFactory; import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor; +import org.apache.flink.runtime.dispatcher.cleanup.DispatcherApplicationResourceCleanerFactory; import org.apache.flink.runtime.dispatcher.cleanup.DispatcherResourceCleanerFactory; import org.apache.flink.runtime.dispatcher.cleanup.TestingRetryStrategies; import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.highavailability.ApplicationResult; +import org.apache.flink.runtime.highavailability.ApplicationResultEntry; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; +import org.apache.flink.runtime.highavailability.EmbeddedApplicationResultStore; import org.apache.flink.runtime.highavailability.JobResultEntry; import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; @@ -37,6 +46,7 @@ import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; +import org.apache.flink.runtime.jobmanager.ApplicationStore; import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobMasterGateway; @@ -44,11 +54,14 @@ import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.leaderelection.LeaderInformation; import org.apache.flink.runtime.leaderelection.TestingLeaderElection; +import org.apache.flink.runtime.messages.FlinkApplicationNotFoundException; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; 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.TestingApplicationResultStore; +import org.apache.flink.runtime.testutils.TestingApplicationStore; import org.apache.flink.runtime.testutils.TestingExecutionPlanStore; import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.streaming.api.graph.ExecutionPlan; @@ -201,6 +214,7 @@ public void testCleanupThroughRetries() throws Exception { public void testCleanupNotCancellable() throws Exception { final JobGraph jobGraph = createJobGraph(); final JobID jobId = jobGraph.getJobID(); + final ApplicationID applicationId = ApplicationID.fromHexString(jobId.toHexString()); final JobResultStore jobResultStore = new EmbeddedJobResultStore(); jobResultStore @@ -209,6 +223,15 @@ public void testCleanupNotCancellable() throws Exception { .get(); haServices.setJobResultStore(jobResultStore); + final ApplicationResultStore applicationResultStore = new EmbeddedApplicationResultStore(); + applicationResultStore + .createDirtyResultAsync( + new ApplicationResultEntry( + TestingApplicationResultStore.createSuccessfulApplicationResult( + applicationId))) + .get(); + haServices.setApplicationResultStore(applicationResultStore); + // Instantiates JobManagerRunner final CompletableFuture jobManagerRunnerCleanupFuture = new CompletableFuture<>(); final AtomicReference jobManagerRunnerEntry = new AtomicReference<>(); @@ -247,6 +270,7 @@ public void testCleanupNotCancellable() throws Exception { public void testCleanupAfterLeadershipChange() throws Exception { final JobGraph jobGraph = createJobGraph(); final JobID jobId = jobGraph.getJobID(); + final ApplicationID applicationId = ApplicationID.fromHexString(jobId.toHexString()); // Construct execution plan store. final AtomicInteger actualGlobalCleanupCallCount = new AtomicInteger(); @@ -311,11 +335,19 @@ public void testCleanupAfterLeadershipChange() throws Exception { .collect(Collectors.toSet())) .as("The JobResultStore should have this job marked as dirty.") .containsExactly(jobId); + assertThat( + haServices.getApplicationResultStore().getDirtyResults().stream() + .map(ApplicationResult::getApplicationId) + .collect(Collectors.toSet())) + .as("The ApplicationResultStore should have this application marked as dirty.") + .containsExactly(applicationId); // Run a second dispatcher, that restores our finished job. final Dispatcher secondDispatcher = createTestingDispatcherBuilder() .setRecoveredDirtyJobs(haServices.getJobResultStore().getDirtyResults()) + .setRecoveredDirtyApplications( + haServices.getApplicationResultStore().getDirtyResults()) .build(rpcService); secondDispatcher.start(); @@ -337,6 +369,200 @@ public void testCleanupAfterLeadershipChange() throws Exception { assertThat(actualGlobalCleanupCallCount.get()).isEqualTo(2); } + @Test + public void testApplicationCleanupThroughRetries() throws Exception { + final ApplicationID applicationId = new ApplicationID(); + final AbstractApplication application = + TestingApplication.builder().setApplicationId(applicationId).build(); + + final AtomicInteger actualGlobalCleanupCallCount = new AtomicInteger(); + final OneShotLatch successfulCleanupLatch = new OneShotLatch(); + final int numberOfErrors = 5; + final RuntimeException temporaryError = + new RuntimeException("Expected RuntimeException: Unable to clean application."); + final AtomicInteger failureCount = new AtomicInteger(numberOfErrors); + final ApplicationStore applicationStore = + TestingApplicationStore.newBuilder() + .setGlobalCleanupFunction( + (ignoredId, ignoredExecutor) -> { + actualGlobalCleanupCallCount.incrementAndGet(); + + if (failureCount.getAndDecrement() > 0) { + return FutureUtils.completedExceptionally(temporaryError); + } + + successfulCleanupLatch.trigger(); + return FutureUtils.completedVoidFuture(); + }) + .build(); + + applicationStore.start(); + haServices.setApplicationStore(applicationStore); + + // start the dispatcher with enough retries on cleanup + final TestingDispatcher dispatcher = + createTestingDispatcherBuilder() + .setApplicationResourceCleanerFactory( + new DispatcherApplicationResourceCleanerFactory( + ForkJoinPool.commonPool(), + TestingRetryStrategies.createWithNumberOfRetries( + numberOfErrors), + haServices.getApplicationStore(), + blobServer)) + .build(rpcService); + dispatcher.start(); + dispatcher.waitUntilStarted(); + + toTerminate.add(dispatcher); + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + dispatcherGateway.submitApplication(application, TIMEOUT).get(); + dispatcher.notifyApplicationStatusChange(applicationId, ApplicationState.FINISHED); + + successfulCleanupLatch.await(); + + assertThat(actualGlobalCleanupCallCount.get()).isEqualTo(numberOfErrors + 1); + + assertThat(haServices.getApplicationStore().getApplicationIds()).isEmpty(); + + CommonTestUtils.waitUntilCondition( + () -> + haServices + .getApplicationResultStore() + .hasApplicationResultEntryAsync(applicationId) + .get()); + } + + @Test + public void testApplicationCleanupNotCancellable() throws Exception { + final ApplicationID applicationId = new ApplicationID(); + + final ApplicationResultStore applicationResultStore = new EmbeddedApplicationResultStore(); + applicationResultStore + .createDirtyResultAsync( + new ApplicationResultEntry( + TestingApplicationResultStore.createSuccessfulApplicationResult( + applicationId))) + .get(); + haServices.setApplicationResultStore(applicationResultStore); + + assertThat( + haServices + .getApplicationResultStore() + .hasDirtyApplicationResultEntryAsync(applicationId) + .get()) + .isTrue(); + + final TestingDispatcher dispatcher = createTestingDispatcherBuilder().build(rpcService); + dispatcher.start(); + dispatcher.waitUntilStarted(); + + toTerminate.add(dispatcher); + + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + + ArchivedApplication application = + dispatcherGateway.requestApplication(applicationId, TIMEOUT).get(); + + assertThat(application.getApplicationId()).isEqualTo(applicationId); + assertThat(application.getApplicationStatus()).isEqualTo(ApplicationState.FINISHED); + + assertThatThrownBy(() -> dispatcherGateway.cancelApplication(applicationId, TIMEOUT).get()) + .hasCauseInstanceOf(FlinkApplicationNotFoundException.class); + + CommonTestUtils.waitUntilCondition( + () -> + haServices + .getApplicationResultStore() + .hasCleanApplicationResultEntryAsync(applicationId) + .get()); + } + + @Test + public void testApplicationCleanupAfterLeadershipChange() throws Exception { + final ApplicationID applicationId = new ApplicationID(); + final AbstractApplication application = + TestingApplication.builder().setApplicationId(applicationId).build(); + + // Construct application store. + final AtomicInteger actualGlobalCleanupCallCount = new AtomicInteger(); + final OneShotLatch firstCleanupTriggered = new OneShotLatch(); + final CompletableFuture successfulApplicationCleanup = + new CompletableFuture<>(); + final ApplicationStore applicationStore = + TestingApplicationStore.newBuilder() + .setGlobalCleanupFunction( + (actualId, ignoredExecutor) -> { + final int callCount = + actualGlobalCleanupCallCount.getAndIncrement(); + firstCleanupTriggered.trigger(); + + if (callCount < 1) { + return FutureUtils.completedExceptionally( + new RuntimeException( + "Expected RuntimeException: Unable to remove application.")); + } + + successfulApplicationCleanup.complete(actualId); + return FutureUtils.completedVoidFuture(); + }) + .build(); + + applicationStore.start(); + haServices.setApplicationStore(applicationStore); + + // start the dispatcher with no retries on cleanup + configuration.set( + CleanupOptions.CLEANUP_STRATEGY, + CleanupOptions.NONE_PARAM_VALUES.iterator().next()); + final TestingDispatcher dispatcher = createTestingDispatcherBuilder().build(rpcService); + dispatcher.start(); + dispatcher.waitUntilStarted(); + + toTerminate.add(dispatcher); + final DispatcherGateway dispatcherGateway = + dispatcher.getSelfGateway(DispatcherGateway.class); + dispatcherGateway.submitApplication(application, TIMEOUT).get(); + dispatcher.notifyApplicationStatusChange(applicationId, ApplicationState.FINISHED); + + firstCleanupTriggered.await(); + + assertThat(actualGlobalCleanupCallCount.get()).isOne(); + assertThat(successfulApplicationCleanup.isDone()).isFalse(); + + assertThat( + haServices.getApplicationResultStore().getDirtyResults().stream() + .map(ApplicationResult::getApplicationId) + .collect(Collectors.toSet())) + .containsExactly(applicationId); + + // Run a second dispatcher, that restores our finished application. + final Dispatcher secondDispatcher = + createTestingDispatcherBuilder() + .setRecoveredDirtyApplications( + haServices.getApplicationResultStore().getDirtyResults()) + .build(rpcService); + secondDispatcher.start(); + + toTerminate.add(secondDispatcher); + + CommonTestUtils.waitUntilCondition( + () -> haServices.getApplicationResultStore().getDirtyResults().isEmpty()); + + assertThat(haServices.getApplicationStore().getApplicationIds()).isEmpty(); + assertThat( + haServices + .getApplicationResultStore() + .hasCleanApplicationResultEntryAsync(applicationId) + .get()) + .isTrue(); + + assertThat(successfulApplicationCleanup.get()).isEqualTo(applicationId); + + assertThat(actualGlobalCleanupCallCount.get()).isEqualTo(2); + } + private void waitForJobToFinish( CompletableFuture confirmedLeaderInformation, DispatcherGateway dispatcherGateway, 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 a8ef89144639b..c492d150c5ade 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 @@ -30,8 +30,6 @@ import org.apache.flink.core.failure.FailureEnricher; import org.apache.flink.core.testutils.FlinkAssertions; import org.apache.flink.core.testutils.OneShotLatch; -import org.apache.flink.runtime.application.AbstractApplication; -import org.apache.flink.runtime.application.ArchivedApplication; import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot; @@ -39,7 +37,6 @@ import org.apache.flink.runtime.checkpoint.Checkpoints; import org.apache.flink.runtime.checkpoint.DefaultCheckpointStatsTracker; import org.apache.flink.runtime.checkpoint.metadata.CheckpointMetadata; -import org.apache.flink.runtime.client.DuplicateApplicationSubmissionException; import org.apache.flink.runtime.client.DuplicateJobSubmissionException; import org.apache.flink.runtime.client.JobSubmissionException; import org.apache.flink.runtime.clusterframework.ApplicationStatus; @@ -49,6 +46,8 @@ import org.apache.flink.runtime.executiongraph.ErrorInfo; import org.apache.flink.runtime.executiongraph.JobStatusListener; import org.apache.flink.runtime.heartbeat.HeartbeatServices; +import org.apache.flink.runtime.highavailability.ApplicationResult; +import org.apache.flink.runtime.highavailability.ApplicationResultEntry; import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.JobResultEntry; import org.apache.flink.runtime.highavailability.JobResultStore; @@ -79,7 +78,6 @@ import org.apache.flink.runtime.leaderelection.LeaderElection; import org.apache.flink.runtime.leaderelection.TestingLeaderElection; import org.apache.flink.runtime.messages.Acknowledge; -import org.apache.flink.runtime.messages.FlinkApplicationTerminatedWithoutCancellationException; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.messages.FlinkJobTerminatedWithoutCancellationException; import org.apache.flink.runtime.messages.webmonitor.JobDetails; @@ -99,6 +97,7 @@ import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; import org.apache.flink.runtime.testtasks.NoOpInvokable; import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.testutils.TestingApplicationResultStore; import org.apache.flink.runtime.testutils.TestingExecutionPlanStore; import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; @@ -156,8 +155,6 @@ 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.junit.Assert.assertFalse; -import static org.junit.Assert.assertThrows; /** Test for the {@link Dispatcher} component. */ public class DispatcherTest extends AbstractDispatcherTest { @@ -184,14 +181,6 @@ public void setUp() throws Exception { haServices.setJobMasterLeaderElection(jobId, jobMasterLeaderElection); } - @Nonnull - private TestingDispatcher createAndStartDispatcher( - HeartbeatServices heartbeatServices, TestingHighAvailabilityServices haServices) - throws Exception { - return createAndStartDispatcher( - heartbeatServices, haServices, new JobManagerRunnerWithBlockingJobMasterFactory()); - } - @Nonnull private TestingDispatcher createAndStartDispatcher( HeartbeatServices heartbeatServices, @@ -205,6 +194,7 @@ private TestingDispatcher createAndStartDispatcher( .setJobManagerRunnerFactory(jobManagerRunnerFactory) .setExecutionPlanWriter(haServices.getExecutionPlanStore()) .setJobResultStore(haServices.getJobResultStore()) + .setApplicationResultStore(haServices.getApplicationResultStore()) .build(rpcService); dispatcher.start(); return dispatcher; @@ -245,8 +235,17 @@ public void testJobSubmission() throws Exception { @Test public void testDuplicateJobSubmissionWithGloballyTerminatedButDirtyJob() throws Exception { final JobResult jobResult = - TestingJobResultStore.createJobResult(jobGraph.getJobID(), JobStatus.FINISHED); + TestingJobResultStore.createJobResult(jobId, JobStatus.FINISHED); haServices.getJobResultStore().createDirtyResultAsync(new JobResultEntry(jobResult)).get(); + + // job result must have a corresponding application result + final ApplicationResult applicationResult = + TestingApplicationResultStore.createSuccessfulApplicationResult(applicationId); + haServices + .getApplicationResultStore() + .createDirtyResultAsync(new ApplicationResultEntry(applicationResult)) + .get(); + assertDuplicateJobSubmission(); } @@ -611,74 +610,6 @@ public void testNoHistoryServerArchiveCreatedForSuspendedJob() throws Exception assertThat(archiveAttemptFuture).isNotDone(); } - @Test - public void testApplicationStatusChange_ArchiveNotCalledForNonTerminalStatus() - throws Exception { - final CompletableFuture archiveApplicationFuture = new CompletableFuture<>(); - dispatcher = - createTestingDispatcherBuilder() - .setHistoryServerArchivist( - TestingHistoryServerArchivist.builder() - .setArchiveApplicationFunction( - archivedApplication -> { - archiveApplicationFuture.complete(null); - return CompletableFuture.completedFuture(null); - }) - .build()) - .build(rpcService); - dispatcher.start(); - submitApplicationAndMockApplicationStatusChange(ApplicationState.RUNNING); - // verify that archive application is not called - assertFalse(archiveApplicationFuture.isDone()); - assertFalse(dispatcher.getApplicationTerminationFuture(applicationId).isDone()); - } - - @Test - public void testApplicationStatusChange_ArchiveCalledForTerminalStatus() throws Exception { - final CompletableFuture archiveApplicationFuture = new CompletableFuture<>(); - dispatcher = - createTestingDispatcherBuilder() - .setHistoryServerArchivist( - TestingHistoryServerArchivist.builder() - .setArchiveApplicationFunction( - archivedApplication -> { - archiveApplicationFuture.complete( - archivedApplication.getApplicationId()); - return CompletableFuture.completedFuture(null); - }) - .build()) - .build(rpcService); - dispatcher.start(); - submitApplicationAndMockApplicationStatusChange(ApplicationState.FINISHED); - // verify that archive application is called with the application id - assertEquals(applicationId, archiveApplicationFuture.get()); - dispatcher - .getApplicationTerminationFuture(applicationId) - .get(TIMEOUT.toMillis(), TimeUnit.MILLISECONDS); - } - - @Test - public void testApplicationStatusChange_ThrowsIfDuplicateTerminalStatus() throws Exception { - dispatcher = createTestingDispatcherBuilder().build(rpcService); - dispatcher.start(); - submitApplicationAndMockApplicationStatusChange(ApplicationState.FINISHED); - // wait for archive to complete - dispatcher - .getApplicationTerminationFuture(applicationId) - .get(TIMEOUT.toMillis(), TimeUnit.MILLISECONDS); - assertThrows( - IllegalStateException.class, - () -> - dispatcher.notifyApplicationStatusChange( - applicationId, ApplicationState.FAILED)); - } - - private void submitApplicationAndMockApplicationStatusChange(ApplicationState targetState) - throws Exception { - submitApplication(); - mockApplicationStatusChange(targetState); - } - private void submitApplication() throws Exception { dispatcher .submitApplication( @@ -1038,10 +969,15 @@ public void testJobCleanupWithoutRecoveredJobGraph() throws Exception { final TestingCleanupRunnerFactory cleanupRunnerFactory = new TestingCleanupRunnerFactory(); final OneShotLatch dispatcherBootstrapLatch = new OneShotLatch(); + // job result must have a corresponding application result dispatcher = createTestingDispatcherBuilder() .setJobManagerRunnerFactory(jobManagerRunnerFactory) .setCleanupRunnerFactory(cleanupRunnerFactory) + .setRecoveredDirtyApplications( + Collections.singleton( + TestingApplicationResultStore + .createSuccessfulApplicationResult(applicationId))) .setRecoveredDirtyJobs( Collections.singleton( new JobResult.Builder() @@ -1074,479 +1010,6 @@ public void testJobCleanupWithoutRecoveredJobGraph() throws Exception { .isZero(); } - @Test - public void testApplicationBootstrap() throws Exception { - final OneShotLatch bootstrapLatch = new OneShotLatch(); - final AbstractApplication application = - TestingApplication.builder() - .setApplicationId(applicationId) - .setExecuteFunction( - ignored -> { - bootstrapLatch.trigger(); - return CompletableFuture.completedFuture(Acknowledge.get()); - }) - .build(); - - dispatcher = - createTestingDispatcherBuilder() - .setDispatcherBootstrapFactory( - (ignoredDispatcherGateway, - ignoredScheduledExecutor, - ignoredFatalErrorHandler) -> - new ApplicationBootstrap(application)) - .setJobManagerRunnerFactory( - new TestingJobMasterServiceLeadershipRunnerFactory()) - .build(rpcService); - - dispatcher.start(); - - // ensure that the application execution is triggered - bootstrapLatch.await(); - - assertThat(dispatcher.getApplications().size()).isEqualTo(1); - assertThat(dispatcher.getApplications().keySet()).contains(applicationId); - - final DispatcherGateway dispatcherGateway = - dispatcher.getSelfGateway(DispatcherGateway.class); - dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - - assertThat(application.getJobs().size()).isEqualTo(1); - assertThat(application.getJobs()).contains(jobId); - } - - @Test - public void testApplicationSubmission() throws Exception { - dispatcher = createAndStartDispatcher(heartbeatServices, haServices); - DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); - - final CompletableFuture submittedApplicationFuture = - new CompletableFuture<>(); - final AbstractApplication application = - TestingApplication.builder() - .setApplicationId(applicationId) - .setExecuteFunction( - ignored -> { - submittedApplicationFuture.complete(applicationId); - return CompletableFuture.completedFuture(Acknowledge.get()); - }) - .build(); - - dispatcherGateway.submitApplication(application, TIMEOUT).get(); - - // ensure that the application execution is triggered - assertThat(submittedApplicationFuture).isCompletedWithValue(applicationId); - - ArchivedApplication archivedApplication = - dispatcher.requestApplication(applicationId, TIMEOUT).get(); - assertThat(archivedApplication.getApplicationId()).isEqualTo(applicationId); - } - - @Test - public void testDuplicateApplicationSubmission() throws Exception { - dispatcher = createAndStartDispatcher(heartbeatServices, haServices); - final DispatcherGateway dispatcherGateway = - dispatcher.getSelfGateway(DispatcherGateway.class); - - final AbstractApplication application = - TestingApplication.builder().setApplicationId(applicationId).build(); - // submit application - dispatcherGateway.submitApplication(application, TIMEOUT).get(); - - // duplicate submission - final CompletableFuture submitFuture = - dispatcherGateway.submitApplication(application, TIMEOUT); - assertThatThrownBy(submitFuture::get) - .hasCauseInstanceOf(DuplicateApplicationSubmissionException.class); - } - - @Test - public void testDuplicateApplicationSubmissionIsDetectedOnSimultaneousSubmission() - throws Exception { - dispatcher = createAndStartDispatcher(heartbeatServices, haServices); - final DispatcherGateway dispatcherGateway = - dispatcher.getSelfGateway(DispatcherGateway.class); - - final AbstractApplication application = - TestingApplication.builder().setApplicationId(applicationId).build(); - - final int numThreads = 5; - final CountDownLatch prepareLatch = new CountDownLatch(numThreads); - final OneShotLatch startLatch = new OneShotLatch(); - - final Collection exceptions = Collections.synchronizedList(new ArrayList<>()); - final Collection threads = new ArrayList<>(); - for (int x = 0; x < numThreads; x++) { - threads.add( - new Thread( - () -> { - try { - prepareLatch.countDown(); - startLatch.awaitQuietly(); - dispatcherGateway - .submitApplication(application, TIMEOUT) - .join(); - } catch (Throwable t) { - exceptions.add(t); - } - })); - } - - // start worker threads and trigger submissions - threads.forEach(Thread::start); - prepareLatch.await(); - startLatch.trigger(); - - // wait for the submissions to happen - for (Thread thread : threads) { - thread.join(); - } - - // verify the application was actually submitted - ArchivedApplication archivedApplication = - dispatcher.requestApplication(applicationId, TIMEOUT).get(); - assertThat(archivedApplication.getApplicationId()).isEqualTo(applicationId); - - // verify that all but one submission failed as duplicates - assertThat(exceptions) - .hasSize(numThreads - 1) - .allSatisfy( - t -> - assertThat(t) - .hasCauseInstanceOf( - DuplicateApplicationSubmissionException.class)); - } - - @Test - public void testRecoverJobSuccessfully() throws Exception { - final TestingJobMasterServiceLeadershipRunnerFactory jobManagerRunnerFactory = - new TestingJobMasterServiceLeadershipRunnerFactory(); - - dispatcher = - createTestingDispatcherBuilder() - .setJobManagerRunnerFactory(jobManagerRunnerFactory) - .setRecoveredJobs(Collections.singleton(jobGraph)) - .setDispatcherBootstrapFactory( - (ignoredDispatcherGateway, - ignoredScheduledExecutor, - ignoredFatalErrorHandler) -> - new ApplicationBootstrap( - TestingApplication.builder() - .setApplicationId(applicationId) - .build())) - .build(rpcService); - - dispatcher.start(); - dispatcher.waitUntilStarted(); - - // verify that the recovered job is NOT recovered immediately - assertThat(jobManagerRunnerFactory.getQueueSize()).isZero(); - assertThat(dispatcher.getSuspendedJobs().containsKey(jobId)).isTrue(); - assertThat(dispatcher.getSuspendedJobIdsByApplicationId().containsKey(applicationId)) - .isTrue(); - - // call recoverJob RPC - final DispatcherGateway dispatcherGateway = - dispatcher.getSelfGateway(DispatcherGateway.class); - dispatcherGateway.recoverJob(jobId, TIMEOUT).get(); - - // verify that the job is now recovered - final TestingJobManagerRunner jobManagerRunner = - jobManagerRunnerFactory.takeCreatedJobManagerRunner(); - assertThat(jobManagerRunner.getJobID()).isEqualTo(jobId); - assertThat(dispatcher.getSuspendedJobs().containsKey(jobId)).isFalse(); - assertThat(dispatcher.getSuspendedJobIdsByApplicationId().containsKey(applicationId)) - .isFalse(); - } - - @Test - public void testRecoverJobFailsWhenJobNotFound() throws Exception { - dispatcher = - createTestingDispatcherBuilder() - .setRecoveredJobs(Collections.singleton(jobGraph)) - .setDispatcherBootstrapFactory( - (ignoredDispatcherGateway, - ignoredScheduledExecutor, - ignoredFatalErrorHandler) -> - new ApplicationBootstrap( - TestingApplication.builder() - .setApplicationId(applicationId) - .build())) - .build(rpcService); - - dispatcher.start(); - dispatcher.waitUntilStarted(); - - // try to recover a job that doesn't exist - final JobID unknownJobId = new JobID(); - final DispatcherGateway dispatcherGateway = - dispatcher.getSelfGateway(DispatcherGateway.class); - final CompletableFuture recoverFuture = - dispatcherGateway.recoverJob(unknownJobId, TIMEOUT); - - assertThatThrownBy(recoverFuture::get) - .hasCauseInstanceOf(JobSubmissionException.class) - .hasMessageContaining("Cannot find the recovered job"); - } - - @Test - public void testRemainingRecoveredJobsCleanedWhenApplicationReachesTerminalState() - throws Exception { - final TestingJobMasterServiceLeadershipRunnerFactory jobManagerRunnerFactory = - new TestingJobMasterServiceLeadershipRunnerFactory(); - final TestingCleanupRunnerFactory cleanupRunnerFactory = new TestingCleanupRunnerFactory(); - - dispatcher = - createTestingDispatcherBuilder() - .setJobManagerRunnerFactory(jobManagerRunnerFactory) - .setCleanupRunnerFactory(cleanupRunnerFactory) - .setRecoveredJobs(Collections.singleton(jobGraph)) - .setDispatcherBootstrapFactory( - (ignoredDispatcherGateway, - ignoredScheduledExecutor, - ignoredFatalErrorHandler) -> - new ApplicationBootstrap( - TestingApplication.builder() - .setApplicationId(applicationId) - .build())) - .build(rpcService); - - dispatcher.start(); - dispatcher.waitUntilStarted(); - - // verify that the recovered job exists - assertThat(dispatcher.getSuspendedJobs().containsKey(jobId)).isTrue(); - assertThat(dispatcher.getSuspendedJobIdsByApplicationId().containsKey(applicationId)) - .isTrue(); - - // complete the application - this should trigger cleanup of the remaining recovered job - dispatcher - .callAsyncInMainThread( - () -> { - dispatcher.notifyApplicationStatusChange( - applicationId, ApplicationState.FINISHED); - return CompletableFuture.completedFuture(Acknowledge.get()); - }) - .get(); - - // verify that no jobs are recovered - assertThat(jobManagerRunnerFactory.getQueueSize()).isZero(); - - // verify that the remaining recovered job is cleaned up - final TestingJobManagerRunner cleanupRunner = - cleanupRunnerFactory.takeCreatedJobManagerRunner(); - assertThat(cleanupRunner.getJobID()).isEqualTo(jobId); - assertThat(dispatcher.getSuspendedJobs().containsKey(jobId)).isFalse(); - assertThat(dispatcher.getSuspendedJobIdsByApplicationId().containsKey(applicationId)) - .isFalse(); - } - - @Test - public void testJobResultNotMarkedCleanUntilApplicationTerminates() throws Exception { - final TestingJobMasterServiceLeadershipRunnerFactory jobManagerRunnerFactory = - new TestingJobMasterServiceLeadershipRunnerFactory(); - - dispatcher = - createTestingDispatcherBuilder() - .setJobManagerRunnerFactory(jobManagerRunnerFactory) - .setDispatcherBootstrapFactory( - (ignoredDispatcherGateway, - ignoredScheduledExecutor, - ignoredFatalErrorHandler) -> - new ApplicationBootstrap( - TestingApplication.builder() - .setApplicationId(applicationId) - .build())) - .build(rpcService); - - dispatcher.start(); - dispatcher.waitUntilStarted(); - - // submit a job - final DispatcherGateway dispatcherGateway = - dispatcher.getSelfGateway(DispatcherGateway.class); - dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - - // complete the job - final TestingJobManagerRunner jobManagerRunner = - jobManagerRunnerFactory.takeCreatedJobManagerRunner(); - final ExecutionGraphInfo completedExecutionGraphInfo = - new ExecutionGraphInfo( - new ArchivedExecutionGraphBuilder() - .setJobID(jobId) - .setState(JobStatus.FINISHED) - .setApplicationId(applicationId) - .build()); - jobManagerRunner.completeResultFuture(completedExecutionGraphInfo); - - // job termination future should not be completed - assertThatThrownBy( - () -> - dispatcher - .getJobTerminationFuture(jobId, TIMEOUT) - .get(10L, TimeUnit.MILLISECONDS)) - .isInstanceOf(TimeoutException.class); - - // verify that the job result is NOT marked clean yet - assertThat( - haServices.getJobResultStore().getDirtyResults().stream() - .anyMatch(r -> r.getJobId().equals(jobId))) - .isTrue(); - - // complete the application - dispatcher.notifyApplicationStatusChange(applicationId, ApplicationState.FINISHED); - - // wait for application termination - dispatcher - .getApplicationTerminationFuture(applicationId) - .get(TIMEOUT.toMillis(), TimeUnit.MILLISECONDS); - - // wait for job termination - dispatcher - .getJobTerminationFuture(jobId, TIMEOUT) - .get(TIMEOUT.toMillis(), TimeUnit.MILLISECONDS); - - // verify that the job result is now marked clean - assertThat( - haServices.getJobResultStore().getDirtyResults().stream() - .noneMatch(r -> r.getJobId().equals(jobId))) - .isTrue(); - } - - @Test - public void testRecoveredDirtyJobResultsCleanedOnApplicationSubmission() throws Exception { - // create a dirty job result - final JobResult jobResult = - new JobResult.Builder() - .jobId(jobId) - .jobStatus(JobStatus.FINISHED) - .netRuntime(1) - .applicationId(applicationId) - .build(); - - final TestingCleanupRunnerFactory cleanupRunnerFactory = new TestingCleanupRunnerFactory(); - - dispatcher = - createTestingDispatcherBuilder() - .setCleanupRunnerFactory(cleanupRunnerFactory) - .setRecoveredDirtyJobs(Collections.singleton(jobResult)) - .setDispatcherBootstrapFactory( - (ignoredDispatcherGateway, - ignoredScheduledExecutor, - ignoredFatalErrorHandler) -> - new ApplicationBootstrap( - TestingApplication.builder() - .setApplicationId(applicationId) - .build())) - .build(rpcService); - - // verify that the dirty job result exists - assertThat( - dispatcher - .getRecoveredDirtyJobResultsByApplicationId() - .containsKey(applicationId)) - .isTrue(); - - // start application - this should trigger the cleanup of dirty job results - dispatcher.start(); - dispatcher.waitUntilStarted(); - - // verify that the dirty job result was cleaned up - final TestingJobManagerRunner cleanupRunner = - cleanupRunnerFactory.takeCreatedJobManagerRunner(); - assertThat(cleanupRunner.getJobID()).isEqualTo(jobId); - assertThat( - dispatcher - .getRecoveredDirtyJobResultsByApplicationId() - .containsKey(applicationId)) - .isFalse(); - } - - @Test - public void testApplicationCancellation() throws Exception { - dispatcher = createAndStartDispatcher(heartbeatServices, haServices); - final DispatcherGateway dispatcherGateway = - dispatcher.getSelfGateway(DispatcherGateway.class); - - final CompletableFuture canceledApplicationFuture = new CompletableFuture<>(); - final AbstractApplication application = - TestingApplication.builder() - .setApplicationId(applicationId) - .setCancelFunction( - ignored -> { - canceledApplicationFuture.complete(null); - return null; - }) - .build(); - - dispatcherGateway.submitApplication(application, TIMEOUT).get(); - - // verify the application was actually submitted - ArchivedApplication archivedApplication = - dispatcher.requestApplication(applicationId, TIMEOUT).get(); - assertThat(archivedApplication.getApplicationId()).isEqualTo(applicationId); - - // submission has succeeded, now cancel the application - dispatcherGateway.cancelApplication(applicationId, TIMEOUT).get(); - - assertThatFuture(canceledApplicationFuture).isDone(); - } - - @Test - public void testApplicationCancellationOfCanceledTerminalDoesNotThrowException() - throws Exception { - dispatcher = createAndStartDispatcher(heartbeatServices, haServices); - final DispatcherGateway dispatcherGateway = - dispatcher.getSelfGateway(DispatcherGateway.class); - - final AbstractApplication application = - TestingApplication.builder() - .setApplicationId(applicationId) - .setGetApplicationStatusFunction(ignored -> ApplicationState.CANCELED) - .build(); - - dispatcherGateway.submitApplication(application, TIMEOUT).get(); - - // verify the application was actually submitted - ArchivedApplication archivedApplication = - dispatcher.requestApplication(applicationId, TIMEOUT).get(); - assertThat(archivedApplication.getApplicationId()).isEqualTo(applicationId); - assertThat(archivedApplication.getApplicationStatus()).isEqualTo(ApplicationState.CANCELED); - - // cancel the application should not throw - dispatcherGateway.cancelApplication(applicationId, TIMEOUT).get(); - } - - @Test - public void testApplicationCancellationOfNonCanceledTerminalFailsWithAppropriateException() - throws Exception { - - dispatcher = createAndStartDispatcher(heartbeatServices, haServices); - final DispatcherGateway dispatcherGateway = - dispatcher.getSelfGateway(DispatcherGateway.class); - - final AbstractApplication application = - TestingApplication.builder() - .setApplicationId(applicationId) - .setGetApplicationStatusFunction(ignored -> ApplicationState.FINISHED) - .build(); - - dispatcherGateway.submitApplication(application, TIMEOUT).get(); - - // verify the application was actually submitted - ArchivedApplication archivedApplication = - dispatcher.requestApplication(applicationId, TIMEOUT).get(); - assertThat(archivedApplication.getApplicationId()).isEqualTo(applicationId); - assertThat(archivedApplication.getApplicationStatus()).isEqualTo(ApplicationState.FINISHED); - - // cancel the application should throw - final CompletableFuture cancelFuture = - dispatcherGateway.cancelApplication(applicationId, TIMEOUT); - - FlinkAssertions.assertThatFuture(cancelFuture) - .eventuallyFails() - .withCauseOfType(FlinkApplicationTerminatedWithoutCancellationException.class); - } - @Test public void testPersistedJobGraphWhenDispatcherIsShutDown() throws Exception { final TestingExecutionPlanStore submittedExecutionPlanStore = @@ -1683,33 +1146,6 @@ public void testShutDownFutureCompletesAfterJobTerminationFutures() throws Excep dispatcher.getShutDownFuture().get(); } - @Test - public void testShutDownFutureCompletesAfterApplicationArchivingFutures() throws Exception { - final CompletableFuture archiveApplicationFuture = new CompletableFuture<>(); - dispatcher = - createTestingDispatcherBuilder() - .setHistoryServerArchivist( - TestingHistoryServerArchivist.builder() - .setArchiveApplicationFunction( - archivedApplication -> archiveApplicationFuture) - .build()) - .build(rpcService); - dispatcher.start(); - - submitApplicationAndMockApplicationStatusChange(ApplicationState.FINISHED); - - final DispatcherGateway dispatcherGateway = - dispatcher.getSelfGateway(DispatcherGateway.class); - - dispatcherGateway.shutDownCluster(ApplicationStatus.SUCCEEDED).get(); - assertThatThrownBy(() -> dispatcher.getShutDownFuture().get(100L, TimeUnit.MILLISECONDS)) - .isInstanceOf(TimeoutException.class); - - archiveApplicationFuture.complete(null); - - dispatcher.getShutDownFuture().get(); - } - @Test public void testOnRemovedJobGraphDoesNotCleanUpHAFiles() throws Exception { final CompletableFuture removeJobGraphFuture = new CompletableFuture<>(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/NoOpApplicationWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/NoOpApplicationWriter.java new file mode 100644 index 0000000000000..0d7430eca612f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/NoOpApplicationWriter.java @@ -0,0 +1,32 @@ +/* + * 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.runtime.jobmanager.ApplicationStoreEntry; +import org.apache.flink.runtime.jobmanager.ApplicationWriter; + +/** Testing implementation of {@link ApplicationWriter} which does nothing. */ +public enum NoOpApplicationWriter implements ApplicationWriter { + INSTANCE; + + @Override + public void putApplication(ApplicationStoreEntry application) { + // No-op. + } +} 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 36efcf9de9531..4d09bb23bf9a8 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,18 +20,25 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.application.AbstractApplication; import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.dispatcher.cleanup.ApplicationResourceCleanerFactory; import org.apache.flink.runtime.dispatcher.cleanup.CleanupRunnerFactory; +import org.apache.flink.runtime.dispatcher.cleanup.DispatcherApplicationResourceCleanerFactory; 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.dispatcher.cleanup.TestingRetryStrategies; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.heartbeat.HeartbeatServicesImpl; +import org.apache.flink.runtime.highavailability.ApplicationResult; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; +import org.apache.flink.runtime.highavailability.EmbeddedApplicationResultStore; 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.jobmanager.ApplicationWriter; import org.apache.flink.runtime.jobmanager.ExecutionPlanWriter; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup; @@ -70,6 +77,8 @@ private TestingDispatcher( DispatcherId fencingToken, Collection recoveredJobs, Collection recoveredDirtyJobs, + Collection recoveredApplications, + Collection recoveredDirtyApplicationResults, Configuration configuration, HighAvailabilityServices highAvailabilityServices, GatewayRetriever resourceManagerGatewayRetriever, @@ -78,6 +87,8 @@ private TestingDispatcher( FatalErrorHandler fatalErrorHandler, ExecutionPlanWriter executionPlanWriter, JobResultStore jobResultStore, + ApplicationWriter applicationWriter, + ApplicationResultStore applicationResultStore, JobManagerMetricGroup jobManagerMetricGroup, @Nullable String metricServiceQueryAddress, Executor ioExecutor, @@ -88,13 +99,16 @@ private TestingDispatcher( DispatcherBootstrapFactory dispatcherBootstrapFactory, DispatcherOperationCaches dispatcherOperationCaches, JobManagerRunnerRegistry jobManagerRunnerRegistry, - ResourceCleanerFactory resourceCleanerFactory) + ResourceCleanerFactory resourceCleanerFactory, + ApplicationResourceCleanerFactory applicationResourceCleanerFactory) throws Exception { super( rpcService, fencingToken, recoveredJobs, recoveredDirtyJobs, + recoveredApplications, + recoveredDirtyApplicationResults, dispatcherBootstrapFactory, new DispatcherServices( configuration, @@ -110,12 +124,15 @@ private TestingDispatcher( jobManagerMetricGroup, executionPlanWriter, jobResultStore, + applicationWriter, + applicationResultStore, jobManagerRunnerFactory, cleanupRunnerFactory, ioExecutor, Collections.emptySet()), jobManagerRunnerRegistry, - resourceCleanerFactory); + resourceCleanerFactory, + applicationResourceCleanerFactory); this.startFuture = new CompletableFuture<>(); } @@ -169,6 +186,8 @@ public static class Builder { private DispatcherId fencingToken = DispatcherId.generate(); private Collection recoveredJobs = Collections.emptyList(); @Nullable private Collection recoveredDirtyJobs = null; + private Collection recoveredApplications = Collections.emptyList(); + @Nullable private Collection recoveredDirtyApplications = null; private HighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices(); @@ -181,6 +200,10 @@ public static class Builder { private ExecutionPlanWriter executionPlanWriter = NoOpExecutionPlanWriter.INSTANCE; private JobResultStore jobResultStore = new EmbeddedJobResultStore(); + private ApplicationWriter applicationWriter = NoOpApplicationWriter.INSTANCE; + private ApplicationResultStore applicationResultStore = + new EmbeddedApplicationResultStore(); + private Configuration configuration = new Configuration(); // even-though it's labeled as @Nullable, it's a mandatory field that needs to be set before @@ -204,6 +227,7 @@ public static class Builder { private JobManagerRunnerRegistry jobManagerRunnerRegistry = new DefaultJobManagerRunnerRegistry(1); @Nullable private ResourceCleanerFactory resourceCleanerFactory; + @Nullable private ApplicationResourceCleanerFactory applicationResourceCleanerFactory; public Builder setFencingToken(DispatcherId fencingToken) { this.fencingToken = fencingToken; @@ -220,6 +244,18 @@ public Builder setRecoveredDirtyJobs(@Nullable Collection recoveredDi return this; } + public Builder setRecoveredApplications( + Collection recoveredApplications) { + this.recoveredApplications = recoveredApplications; + return this; + } + + public Builder setRecoveredDirtyApplications( + @Nullable Collection recoveredDirtyApplications) { + this.recoveredDirtyApplications = recoveredDirtyApplications; + return this; + } + public Builder setHighAvailabilityServices( HighAvailabilityServices highAvailabilityServices) { this.highAvailabilityServices = highAvailabilityServices; @@ -253,6 +289,16 @@ public Builder setJobResultStore(JobResultStore jobResultStore) { return this; } + public Builder setApplicationWriter(ApplicationWriter applicationWriter) { + this.applicationWriter = applicationWriter; + return this; + } + + public Builder setApplicationResultStore(ApplicationResultStore applicationResultStore) { + this.applicationResultStore = applicationResultStore; + return this; + } + public Builder setConfiguration(Configuration configuration) { this.configuration = configuration; return this; @@ -327,6 +373,12 @@ public Builder setResourceCleanerFactory(ResourceCleanerFactory resourceCleanerF return this; } + public Builder setApplicationResourceCleanerFactory( + ApplicationResourceCleanerFactory applicationResourceCleanerFactory) { + this.applicationResourceCleanerFactory = applicationResourceCleanerFactory; + return this; + } + private ResourceCleanerFactory createDefaultResourceCleanerFactory() { return new DispatcherResourceCleanerFactory( ioExecutor, @@ -338,6 +390,14 @@ private ResourceCleanerFactory createDefaultResourceCleanerFactory() { jobManagerMetricGroup); } + private ApplicationResourceCleanerFactory createDefaultApplicationResourceCleanerFactory() { + return new DispatcherApplicationResourceCleanerFactory( + ioExecutor, + TestingRetryStrategies.NO_RETRY_STRATEGY, + applicationWriter, + blobServer); + } + public TestingDispatcher build(RpcService rpcService) throws Exception { return new TestingDispatcher( rpcService, @@ -346,6 +406,10 @@ public TestingDispatcher build(RpcService rpcService) throws Exception { recoveredDirtyJobs == null ? jobResultStore.getDirtyResults() : recoveredDirtyJobs, + recoveredApplications, + recoveredDirtyApplications == null + ? applicationResultStore.getDirtyResults() + : recoveredDirtyApplications, configuration, highAvailabilityServices, resourceManagerGatewayRetriever, @@ -356,6 +420,8 @@ public TestingDispatcher build(RpcService rpcService) throws Exception { fatalErrorHandler, executionPlanWriter, jobResultStore, + applicationWriter, + applicationResultStore, jobManagerMetricGroup, metricServiceQueryAddress, ioExecutor, @@ -368,7 +434,10 @@ public TestingDispatcher build(RpcService rpcService) throws Exception { jobManagerRunnerRegistry, resourceCleanerFactory != null ? resourceCleanerFactory - : createDefaultResourceCleanerFactory()); + : createDefaultResourceCleanerFactory(), + applicationResourceCleanerFactory != null + ? applicationResourceCleanerFactory + : createDefaultApplicationResourceCleanerFactory()); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultApplicationResourceCleanerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultApplicationResourceCleanerTest.java new file mode 100644 index 0000000000000..9715efb90ad3a --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DefaultApplicationResourceCleanerTest.java @@ -0,0 +1,256 @@ +/* + * 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.ApplicationID; +import org.apache.flink.core.testutils.FlinkAssertions; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.TestLoggerExtension; +import org.apache.flink.util.concurrent.Executors; +import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.concurrent.RetryStrategy; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiFunction; +import java.util.function.Consumer; + +import static org.apache.flink.core.testutils.FlinkAssertions.STREAM_THROWABLE; +import static org.apache.flink.core.testutils.FlinkAssertions.assertThatFuture; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * {@code DefaultApplicationResourceCleanerTest} tests {@link DefaultApplicationResourceCleaner}. + */ +@ExtendWith(TestLoggerExtension.class) +class DefaultApplicationResourceCleanerTest { + + private static final Executor EXECUTOR = Executors.directExecutor(); + private static final ApplicationID APPLICATION_ID = new ApplicationID(); + + @Test + void testSuccessfulConcurrentCleanup() { + final SingleCallCleanup cleanup0 = SingleCallCleanup.withoutCompletionOnCleanup(); + final SingleCallCleanup cleanup1 = SingleCallCleanup.withoutCompletionOnCleanup(); + + final CompletableFuture cleanupResult = + createTestInstanceBuilder() + .withRegularCleanup("Reg #0", cleanup0) + .withRegularCleanup("Reg #1", cleanup1) + .build() + .cleanupAsync(APPLICATION_ID); + + assertThat(cleanupResult).isNotCompleted(); + assertThat(cleanup0) + .extracting(SingleCallCleanup::getProcessedApplicationId) + .isEqualTo(APPLICATION_ID); + assertThat(cleanup1) + .extracting(SingleCallCleanup::getProcessedApplicationId) + .isEqualTo(APPLICATION_ID); + + cleanup0.completeCleanup(); + assertThat(cleanupResult).isNotCompleted(); + + cleanup1.completeCleanup(); + assertThat(cleanupResult).isCompleted(); + } + + @Test + void testConcurrentCleanupWithExceptionFirst() { + final SingleCallCleanup cleanup0 = SingleCallCleanup.withoutCompletionOnCleanup(); + final SingleCallCleanup cleanup1 = SingleCallCleanup.withoutCompletionOnCleanup(); + + final CompletableFuture cleanupResult = + createTestInstanceBuilder() + .withRegularCleanup("Reg #0", cleanup0) + .withRegularCleanup("Reg #1", cleanup1) + .build() + .cleanupAsync(APPLICATION_ID); + + assertThat(cleanupResult).isNotCompleted(); + assertThat(cleanup0) + .extracting(SingleCallCleanup::getProcessedApplicationId) + .isEqualTo(APPLICATION_ID); + assertThat(cleanup1) + .extracting(SingleCallCleanup::getProcessedApplicationId) + .isEqualTo(APPLICATION_ID); + + final RuntimeException expectedException = new RuntimeException("Expected exception"); + cleanup0.completeCleanupExceptionally(expectedException); + assertThat(cleanupResult).isNotCompleted(); + + cleanup1.completeCleanup(); + assertThatFuture(cleanupResult) + .eventuallyFailsWith(ExecutionException.class) + .extracting(FlinkAssertions::chainOfCauses, STREAM_THROWABLE) + .hasExactlyElementsOfTypes( + ExecutionException.class, + FutureUtils.RetryException.class, + CompletionException.class, + expectedException.getClass()) + .last() + .isEqualTo(expectedException); + } + + @Test + void testConcurrentCleanupWithExceptionSecond() { + final SingleCallCleanup cleanup0 = SingleCallCleanup.withoutCompletionOnCleanup(); + final SingleCallCleanup cleanup1 = SingleCallCleanup.withoutCompletionOnCleanup(); + + final CompletableFuture cleanupResult = + createTestInstanceBuilder() + .withRegularCleanup("Reg #0", cleanup0) + .withRegularCleanup("Reg #1", cleanup1) + .build() + .cleanupAsync(APPLICATION_ID); + + assertThat(cleanupResult).isNotCompleted(); + assertThat(cleanup0) + .extracting(SingleCallCleanup::getProcessedApplicationId) + .isEqualTo(APPLICATION_ID); + assertThat(cleanup1) + .extracting(SingleCallCleanup::getProcessedApplicationId) + .isEqualTo(APPLICATION_ID); + + cleanup0.completeCleanup(); + assertThat(cleanupResult).isNotCompleted(); + + final RuntimeException expectedException = new RuntimeException("Expected exception"); + cleanup1.completeCleanupExceptionally(expectedException); + assertThatFuture(cleanupResult) + .eventuallyFailsWith(ExecutionException.class) + .extracting(FlinkAssertions::chainOfCauses, STREAM_THROWABLE) + .hasExactlyElementsOfTypes( + ExecutionException.class, + FutureUtils.RetryException.class, + CompletionException.class, + expectedException.getClass()) + .last() + .isEqualTo(expectedException); + } + + @Test + void testCleanupWithRetries() { + final Collection actualApplicationIds = new ArrayList<>(); + final CleanupCallback cleanupWithRetries = + cleanupWithInitialFailingRuns(actualApplicationIds, 2); + final SingleCallCleanup oneRunCleanup = SingleCallCleanup.withCompletionOnCleanup(); + + final CompletableFuture compositeCleanupResult = + createTestInstanceBuilder(TestingRetryStrategies.createWithNumberOfRetries(2)) + .withRegularCleanup("Reg #0", cleanupWithRetries) + .withRegularCleanup("Reg #1", oneRunCleanup) + .build() + .cleanupAsync(APPLICATION_ID); + + assertThatFuture(compositeCleanupResult).eventuallySucceeds(); + + assertThat(oneRunCleanup.getProcessedApplicationId()).isEqualTo(APPLICATION_ID); + assertThat(oneRunCleanup.isDone()).isTrue(); + assertThat(actualApplicationIds) + .containsExactly(APPLICATION_ID, APPLICATION_ID, APPLICATION_ID); + } + + private static DefaultApplicationResourceCleaner.Builder + createTestInstanceBuilder() { + return createTestInstanceBuilder(TestingRetryStrategies.NO_RETRY_STRATEGY); + } + + private static DefaultApplicationResourceCleaner.Builder + createTestInstanceBuilder(RetryStrategy retryStrategy) { + return DefaultApplicationResourceCleaner.forCleanableResources( + ComponentMainThreadExecutorServiceAdapter.forMainThread(), + EXECUTOR, + CleanupCallback::apply, + retryStrategy); + } + + private static CleanupCallback cleanupWithInitialFailingRuns( + Collection actualApplicationIds, int numberOfFailureRuns) { + final AtomicInteger failureRunCount = new AtomicInteger(numberOfFailureRuns); + return (actualApplicationId, executor) -> { + actualApplicationIds.add(actualApplicationId); + if (failureRunCount.getAndDecrement() > 0) { + return FutureUtils.completedExceptionally( + new RuntimeException("Expected RuntimeException")); + } + + return FutureUtils.completedVoidFuture(); + }; + } + + private interface CleanupCallback + extends BiFunction> { + // empty interface to remove necessity use generics all the time + } + + private static class SingleCallCleanup implements CleanupCallback { + + private final CompletableFuture resultFuture = new CompletableFuture<>(); + private ApplicationID applicationId; + + private final Consumer> internalFunction; + + public static SingleCallCleanup withCompletionOnCleanup() { + return new SingleCallCleanup(resultFuture -> resultFuture.complete(null)); + } + + public static SingleCallCleanup withoutCompletionOnCleanup() { + return new SingleCallCleanup(ignoredResultFuture -> {}); + } + + private SingleCallCleanup(Consumer> internalFunction) { + this.internalFunction = internalFunction; + } + + public CompletableFuture apply(ApplicationID applicationId, Executor executor) { + Preconditions.checkState(this.applicationId == null); + this.applicationId = applicationId; + + internalFunction.accept(resultFuture); + + return resultFuture; + } + + public boolean isDone() { + return resultFuture.isDone(); + } + + public ApplicationID getProcessedApplicationId() { + return applicationId; + } + + 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/DispatcherApplicationResourceCleanerFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherApplicationResourceCleanerFactoryTest.java new file mode 100644 index 0000000000000..9c7e417abd956 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/DispatcherApplicationResourceCleanerFactoryTest.java @@ -0,0 +1,136 @@ +/* + * 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.ApplicationID; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.blob.BlobServer; +import org.apache.flink.runtime.blob.TestingBlobStoreBuilder; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; +import org.apache.flink.runtime.jobmanager.ApplicationStoreEntry; +import org.apache.flink.runtime.jobmanager.ApplicationWriter; +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.Executor; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * {@code DispatcherApplicationResourceCleanerFactoryTest} verifies that the resources are properly + * cleaned up for the {@link GloballyCleanableApplicationResource} of the {@link + * org.apache.flink.runtime.dispatcher.Dispatcher} in Application mode. + */ +public class DispatcherApplicationResourceCleanerFactoryTest { + + private static final ApplicationID APPLICATION_ID = new ApplicationID(); + + private CleanableBlobServer blobServer; + private CleanableApplicationWriter applicationWriter; + + private DispatcherApplicationResourceCleanerFactory testInstance; + + @BeforeEach + public void setup() throws Exception { + blobServer = new CleanableBlobServer(); + applicationWriter = new CleanableApplicationWriter(); + + testInstance = + new DispatcherApplicationResourceCleanerFactory( + Executors.directExecutor(), + TestingRetryStrategies.NO_RETRY_STRATEGY, + applicationWriter, + blobServer); + } + + @Test + public void testApplicationResourceCleaning() { + assertCleanupNotTriggered(); + + final CompletableFuture cleanupResultFuture = + testInstance + .createApplicationResourceCleaner( + ComponentMainThreadExecutorServiceAdapter.forMainThread()) + .cleanupAsync(APPLICATION_ID); + + assertThat(cleanupResultFuture).isCompleted(); + + assertThat(applicationWriter.getGlobalCleanupFuture()).isCompleted(); + assertThat(blobServer.getGlobalCleanupFuture()).isCompleted(); + } + + private void assertCleanupNotTriggered() { + assertThat(applicationWriter.getGlobalCleanupFuture()).isNotDone(); + assertThat(blobServer.getGlobalCleanupFuture()).isNotDone(); + } + + /** Test implementation of {@link ApplicationWriter} that tracks cleanup invocations. */ + private static class CleanableApplicationWriter implements ApplicationWriter { + + private final CompletableFuture globalCleanupFuture = + new CompletableFuture<>(); + + @Override + public void putApplication(ApplicationStoreEntry application) { + // No-op for testing + } + + @Override + public CompletableFuture globalCleanupAsync( + ApplicationID applicationId, Executor ignoredExecutor) { + globalCleanupFuture.complete(applicationId); + return FutureUtils.completedVoidFuture(); + } + + public CompletableFuture getGlobalCleanupFuture() { + return globalCleanupFuture; + } + } + + /** Test implementation of {@link BlobServer} that tracks cleanup invocations. */ + private static class CleanableBlobServer extends BlobServer { + + private final CompletableFuture globalCleanupFuture = + new CompletableFuture<>(); + + public CleanableBlobServer() throws IOException { + super( + new Configuration(), + new File("non-existent-file"), + new TestingBlobStoreBuilder().createTestingBlobStore()); + } + + @Override + public CompletableFuture globalCleanupAsync( + ApplicationID applicationId, Executor ignoredExecutor) { + globalCleanupFuture.complete(applicationId); + return FutureUtils.completedVoidFuture(); + } + + public CompletableFuture getGlobalCleanupFuture() { + return globalCleanupFuture; + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingApplicationResourceCleanerFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingApplicationResourceCleanerFactory.java new file mode 100644 index 0000000000000..24f7314515990 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingApplicationResourceCleanerFactory.java @@ -0,0 +1,100 @@ +/* + * 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.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.concurrent.Executors; +import org.apache.flink.util.concurrent.FutureUtils; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.Executor; + +/** + * {@code TestingApplicationResourceCleanerFactory} for adding custom {@link + * ApplicationResourceCleaner} creation. + */ +public class TestingApplicationResourceCleanerFactory implements ApplicationResourceCleanerFactory { + + private final Collection globallyCleanableResources; + + private final Executor cleanupExecutor; + + private TestingApplicationResourceCleanerFactory( + Collection globallyCleanableResources, + Executor cleanupExecutor) { + this.globallyCleanableResources = globallyCleanableResources; + this.cleanupExecutor = cleanupExecutor; + } + + @Override + public ApplicationResourceCleaner createApplicationResourceCleaner( + ComponentMainThreadExecutor mainThreadExecutor) { + return applicationId -> { + mainThreadExecutor.assertRunningInMainThread(); + Throwable t = null; + for (GloballyCleanableApplicationResource resource : globallyCleanableResources) { + try { + resource.globalCleanupAsync(applicationId, cleanupExecutor).get(); + } catch (Throwable throwable) { + t = ExceptionUtils.firstOrSuppressed(throwable, t); + } + } + return t != null + ? FutureUtils.completedExceptionally(t) + : FutureUtils.completedVoidFuture(); + }; + } + + public static Builder builder() { + return new Builder(); + } + + /** {@code Builder} for creating {@code TestingApplicationResourceCleanerFactory} instances. */ + public static class Builder { + + private Collection globallyCleanableResources = + new ArrayList<>(); + + private Executor cleanupExecutor = Executors.directExecutor(); + + public Builder setGloballyCleanableResources( + Collection globallyCleanableResources) { + this.globallyCleanableResources = globallyCleanableResources; + return this; + } + + public Builder withGloballyCleanableResource( + GloballyCleanableApplicationResource globallyCleanableResource) { + this.globallyCleanableResources.add(globallyCleanableResource); + return this; + } + + public Builder setCleanupExecutor(Executor cleanupExecutor) { + this.cleanupExecutor = cleanupExecutor; + return this; + } + + public TestingApplicationResourceCleanerFactory build() { + return new TestingApplicationResourceCleanerFactory( + globallyCleanableResources, cleanupExecutor); + } + } +} 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 ab2b4f595d02a..87ae1845c0f9b 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.AllCallbackWrapper; +import org.apache.flink.runtime.application.AbstractApplication; import org.apache.flink.runtime.application.SingleJobApplication; import org.apache.flink.runtime.dispatcher.Dispatcher; import org.apache.flink.runtime.dispatcher.DispatcherBootstrapFactory; @@ -30,25 +31,30 @@ import org.apache.flink.runtime.dispatcher.DispatcherServices; import org.apache.flink.runtime.dispatcher.JobManagerRunnerFactory; import org.apache.flink.runtime.dispatcher.PartialDispatcherServices; -import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithJobPersistenceComponents; +import org.apache.flink.runtime.dispatcher.PartialDispatcherServicesWithPersistenceComponents; import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; import org.apache.flink.runtime.dispatcher.StandaloneDispatcher; 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.ApplicationResult; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; +import org.apache.flink.runtime.highavailability.EmbeddedApplicationResultStore; 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.ApplicationStore; import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; -import org.apache.flink.runtime.jobmanager.TestingJobPersistenceComponentFactory; +import org.apache.flink.runtime.jobmanager.TestingPersistenceComponentFactory; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.jobmaster.TestingJobManagerRunner; import org.apache.flink.runtime.leaderelection.LeaderInformation; import org.apache.flink.runtime.leaderelection.TestingLeaderElection; import org.apache.flink.runtime.rpc.RpcService; import org.apache.flink.runtime.rpc.TestingRpcServiceExtension; +import org.apache.flink.runtime.testutils.TestingApplicationStore; import org.apache.flink.runtime.testutils.TestingExecutionPlanStore; import org.apache.flink.runtime.util.BlobServerExtension; import org.apache.flink.runtime.util.TestingFatalErrorHandler; @@ -104,6 +110,10 @@ class DefaultDispatcherRunnerITCase { private JobResultStore jobResultStore; + private ApplicationStore applicationStore; + + private ApplicationResultStore applicationResultStore; + private PartialDispatcherServices partialDispatcherServices; private DefaultDispatcherRunnerFactory dispatcherRunnerFactory; @@ -119,6 +129,8 @@ void setup() { fatalErrorHandler = new TestingFatalErrorHandler(); executionPlanStore = TestingExecutionPlanStore.newBuilder().build(); jobResultStore = new EmbeddedJobResultStore(); + applicationStore = TestingApplicationStore.newBuilder().build(); + applicationResultStore = new EmbeddedApplicationResultStore(); partialDispatcherServices = TestingPartialDispatcherServices.builder() @@ -257,18 +269,22 @@ public Dispatcher createDispatcher( DispatcherId fencingToken, Collection recoveredJobs, Collection recoveredDirtyJobResults, + Collection recoveredApplications, + Collection recoveredDirtyApplicationResults, DispatcherBootstrapFactory dispatcherBootstrapFactory, - PartialDispatcherServicesWithJobPersistenceComponents - partialDispatcherServicesWithJobPersistenceComponents) + PartialDispatcherServicesWithPersistenceComponents + partialDispatcherServicesWithPersistenceComponents) throws Exception { return new StandaloneDispatcher( rpcService, fencingToken, recoveredJobs, recoveredDirtyJobResults, + recoveredApplications, + recoveredDirtyApplicationResults, dispatcherBootstrapFactory, DispatcherServices.from( - partialDispatcherServicesWithJobPersistenceComponents, + partialDispatcherServicesWithPersistenceComponents, jobManagerRunnerFactory, cleanupRunnerFactory)); } @@ -282,7 +298,11 @@ private DispatcherRunner createDispatcherRunner() throws Exception { return dispatcherRunnerFactory.createDispatcherRunner( dispatcherLeaderElection, fatalErrorHandler, - new TestingJobPersistenceComponentFactory(executionPlanStore, jobResultStore), + new TestingPersistenceComponentFactory( + executionPlanStore, + jobResultStore, + applicationStore, + applicationResultStore), EXECUTOR_RESOURCE.getExecutor(), rpcServiceExtensionWrapper.getCustomExtension().getTestingRpcService(), partialDispatcherServices); 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 6a3ad40817e2e..99cf2b97205d1 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 @@ -18,17 +18,29 @@ package org.apache.flink.runtime.dispatcher.runner; +import org.apache.flink.api.common.ApplicationID; 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.application.AbstractApplication; +import org.apache.flink.runtime.application.SingleJobApplication; +import org.apache.flink.runtime.blob.BlobServer; import org.apache.flink.runtime.client.JobSubmissionException; +import org.apache.flink.runtime.dispatcher.TestingApplication; +import org.apache.flink.runtime.highavailability.ApplicationResult; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; 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.ApplicationStore; +import org.apache.flink.runtime.jobmanager.ApplicationStoreEntry; import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; +import org.apache.flink.runtime.jobmanager.TestingApplicationStoreEntry; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.messages.Acknowledge; +import org.apache.flink.runtime.messages.FlinkApplicationNotFoundException; +import org.apache.flink.runtime.testutils.TestingApplicationResultStore; +import org.apache.flink.runtime.testutils.TestingApplicationStore; import org.apache.flink.runtime.testutils.TestingExecutionPlanStore; import org.apache.flink.runtime.testutils.TestingJobResultStore; import org.apache.flink.runtime.util.TestingFatalErrorHandler; @@ -75,6 +87,10 @@ class SessionDispatcherLeaderProcessTest { private static ExecutorService ioExecutor; + private static ApplicationStoreEntry applicationStoreEntry; + + private static AbstractApplication application; + private final UUID leaderSessionId = UUID.randomUUID(); private TestingFatalErrorHandler fatalErrorHandler; @@ -82,12 +98,24 @@ class SessionDispatcherLeaderProcessTest { private ExecutionPlanStore executionPlanStore; private JobResultStore jobResultStore; + private ApplicationStore applicationStore; + private ApplicationResultStore applicationResultStore; + + private BlobServer blobServer; + private AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory dispatcherServiceFactory; @BeforeAll static void setupClass() { ioExecutor = Executors.newSingleThreadExecutor(); + + ApplicationID applicationId = + ApplicationID.fromHexString(JOB_GRAPH.getJobID().toHexString()); + application = TestingApplication.builder().setApplicationId(applicationId).build(); + applicationStoreEntry = + TestingApplicationStoreEntry.newBuilder().setApplication(application).build(); + JOB_GRAPH.setApplicationId(applicationId); } @BeforeEach @@ -95,6 +123,8 @@ void setup() { fatalErrorHandler = new TestingFatalErrorHandler(); executionPlanStore = TestingExecutionPlanStore.newBuilder().build(); jobResultStore = TestingJobResultStore.builder().build(); + applicationStore = TestingApplicationStore.newBuilder().build(); + applicationResultStore = TestingApplicationResultStore.builder().build(); dispatcherServiceFactory = createFactoryBasedOnGenericSupplier( () -> TestingDispatcherGatewayService.newBuilder().build()); @@ -175,6 +205,7 @@ void testRecoveryWithMultipleExecutionPlansAndOneMatchingDirtyJobResult() throws final JobResult matchingDirtyJobResult = TestingJobResultStore.createSuccessfulJobResult(JOB_GRAPH.getJobID()); final ExecutionPlan otherExecutionPlan = JobGraphTestUtils.emptyJobGraph(); + otherExecutionPlan.setApplicationId(JOB_GRAPH.getApplicationId().get()); testJobRecovery( Arrays.asList(otherExecutionPlan, JOB_GRAPH), @@ -229,8 +260,12 @@ private void testJobRecovery( (ignoredDispatcherId, recoveredJobs, recoveredDirtyJobResults, + ignoredRecoveredApplications, + ignoredRecoveredDirtyApplicationResults, ignoredExecutionPlanWriter, - ignoredJobResultStore) -> { + ignoredJobResultStore, + ignoredApplicationStore, + ignoredApplicationResultStore) -> { recoveredExecutionPlansFuture.complete(recoveredJobs); recoveredDirtyJobResultsFuture.complete(recoveredDirtyJobResults); return TestingDispatcherGatewayService.newBuilder().build(); @@ -245,6 +280,122 @@ private void testJobRecovery( } } + @Test + void testApplicationRecoveryWithApplicationButNoDirtyApplicationResult() throws Exception { + testApplicationRecovery( + Collections.singleton(applicationStoreEntry), + Collections.emptySet(), + actualRecoveredApplications -> + assertThat(actualRecoveredApplications) + .singleElement() + .isEqualTo(application), + actualRecoveredDirtyApplicationResults -> + assertThat(actualRecoveredDirtyApplicationResults).isEmpty()); + } + + @Test + void testApplicationRecoveryWithApplicationAndMatchingDirtyApplicationResult() + throws Exception { + final ApplicationResult matchingDirtyApplicationResult = + TestingApplicationResultStore.createSuccessfulApplicationResult( + application.getApplicationId()); + + testApplicationRecovery( + Collections.singleton(applicationStoreEntry), + Collections.singleton(matchingDirtyApplicationResult), + actualRecoveredApplications -> assertThat(actualRecoveredApplications).isEmpty(), + actualRecoveredDirtyApplicationResults -> + assertThat(actualRecoveredDirtyApplicationResults) + .singleElement() + .isEqualTo(matchingDirtyApplicationResult)); + } + + @Test + void testApplicationRecoveryWithMultipleApplicationsAndOneMatchingDirtyApplicationResult() + throws Exception { + final ApplicationResult matchingDirtyApplicationResult = + TestingApplicationResultStore.createSuccessfulApplicationResult( + application.getApplicationId()); + final AbstractApplication otherApplication = + TestingApplication.builder().setApplicationId(new ApplicationID()).build(); + final ApplicationStoreEntry otherApplicationEntry = + TestingApplicationStoreEntry.newBuilder().setApplication(otherApplication).build(); + + testApplicationRecovery( + Arrays.asList(otherApplicationEntry, applicationStoreEntry), + Collections.singleton(matchingDirtyApplicationResult), + actualRecoveredApplications -> + assertThat(actualRecoveredApplications) + .singleElement() + .isEqualTo(otherApplication), + actualRecoveredDirtyApplicationResults -> + assertThat(actualRecoveredDirtyApplicationResults) + .singleElement() + .isEqualTo(matchingDirtyApplicationResult)); + } + + @Test + void testApplicationRecoveryWithoutApplicationButDirtyApplicationResult() throws Exception { + final ApplicationResult dirtyApplicationResult = + TestingApplicationResultStore.createSuccessfulApplicationResult( + new ApplicationID()); + + testApplicationRecovery( + Collections.emptyList(), + Collections.singleton(dirtyApplicationResult), + actualRecoveredApplications -> assertThat(actualRecoveredApplications).isEmpty(), + actualRecoveredDirtyApplicationResults -> + assertThat(actualRecoveredDirtyApplicationResults) + .singleElement() + .isEqualTo(dirtyApplicationResult)); + } + + private void testApplicationRecovery( + Collection applicationsToRecover, + Set dirtyApplicationResults, + Consumer> recoveredApplicationAssertion, + Consumer> recoveredDirtyApplicationResultAssertion) + throws Exception { + applicationStore = + TestingApplicationStore.newBuilder() + .setInitialApplications(applicationsToRecover) + .build(); + + applicationResultStore = + TestingApplicationResultStore.builder() + .withGetDirtyResultsSupplier(() -> dirtyApplicationResults) + .build(); + + final CompletableFuture> recoveredApplicationsFuture = + new CompletableFuture<>(); + final CompletableFuture> + recoveredDirtyApplicationResultsFuture = new CompletableFuture<>(); + dispatcherServiceFactory = + (ignoredDispatcherId, + ignoredRecoveredJobs, + ignoredRecoveredDirtyJobResults, + recoveredApplications, + recoveredDirtyApplicationResults, + ignoredExecutionPlanWriter, + ignoredJobResultStore, + ignoredApplicationStore, + ignoredApplicationResultStore) -> { + recoveredApplicationsFuture.complete(recoveredApplications); + recoveredDirtyApplicationResultsFuture.complete( + recoveredDirtyApplicationResults); + return TestingDispatcherGatewayService.newBuilder().build(); + }; + + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = + createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + + recoveredApplicationAssertion.accept(recoveredApplicationsFuture.get()); + recoveredDirtyApplicationResultAssertion.accept( + recoveredDirtyApplicationResultsFuture.get()); + } + } + @Test void testRecoveryWhileExecutionPlanRecoveryIsScheduledConcurrently() throws Exception { final JobResult dirtyJobResult = @@ -282,8 +433,12 @@ void testRecoveryWhileExecutionPlanRecoveryIsScheduledConcurrently() throws Exce (ignoredDispatcherId, recoveredJobs, recoveredDirtyJobResults, + ignoredRecoveredApplications, + ignoredRecoveredDirtyApplicationResults, ignoredExecutionPlanWriter, - ignoredJobResultStore) -> { + ignoredJobResultStore, + ignoredApplicationStore, + ignoredApplicationResultStore) -> { recoveredExecutionPlansFuture.complete(recoveredJobs); recoveredDirtyJobResultsFuture.complete(recoveredDirtyJobResults); return TestingDispatcherGatewayService.newBuilder().build(); @@ -313,11 +468,17 @@ void testRecoveryWhileExecutionPlanRecoveryIsScheduledConcurrently() throws Exce } @Test - void closeAsync_stopsExecutionPlanStoreAndDispatcher() throws Exception { - final CompletableFuture jobGraphStopFuture = new CompletableFuture<>(); + void closeAsync_stopsServicesAndDispatcher() throws Exception { + final CompletableFuture executionPlanStoreStopFuture = new CompletableFuture<>(); executionPlanStore = TestingExecutionPlanStore.newBuilder() - .setStopRunnable(() -> jobGraphStopFuture.complete(null)) + .setStopRunnable(() -> executionPlanStoreStopFuture.complete(null)) + .build(); + + final CompletableFuture applicationStoreStopFuture = new CompletableFuture<>(); + applicationStore = + TestingApplicationStore.newBuilder() + .setStopRunnable(() -> applicationStoreStopFuture.complete(null)) .build(); final CompletableFuture dispatcherServiceTerminationFuture = @@ -339,13 +500,17 @@ void closeAsync_stopsExecutionPlanStoreAndDispatcher() throws Exception { final CompletableFuture terminationFuture = dispatcherLeaderProcess.closeAsync(); - assertThat(jobGraphStopFuture).isNotDone(); + assertThat(executionPlanStoreStopFuture).isNotDone(); + assertThat(applicationStoreStopFuture).isNotDone(); assertThat(terminationFuture).isNotDone(); dispatcherServiceTerminationFuture.complete(null); // verify that we shut down the ExecutionPlanStore - jobGraphStopFuture.get(); + executionPlanStoreStopFuture.get(); + + // verify that we shut down the ApplicationStore + applicationStoreStopFuture.get(); // verify that we completed the dispatcher leader process shut down terminationFuture.get(); @@ -476,6 +641,46 @@ void closeAsync_duringJobRecovery_preventsDispatcherServiceCreation() throws Exc } } + @Test + void closeAsync_duringApplicationRecovery_preventsDispatcherServiceCreation() throws Exception { + final OneShotLatch applicationRecoveryStartedLatch = new OneShotLatch(); + final OneShotLatch completeApplicationRecoveryLatch = new OneShotLatch(); + final OneShotLatch createDispatcherServiceLatch = new OneShotLatch(); + + this.applicationStore = + TestingApplicationStore.newBuilder() + .setApplicationIdsFunction( + storedApplications -> { + applicationRecoveryStartedLatch.trigger(); + completeApplicationRecoveryLatch.await(); + return storedApplications; + }) + .build(); + + this.dispatcherServiceFactory = + createFactoryBasedOnGenericSupplier( + () -> { + createDispatcherServiceLatch.trigger(); + return TestingDispatcherGatewayService.newBuilder().build(); + }); + + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = + createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + + applicationRecoveryStartedLatch.await(); + + dispatcherLeaderProcess.closeAsync(); + + completeApplicationRecoveryLatch.trigger(); + + assertThatThrownBy( + () -> createDispatcherServiceLatch.await(10L, TimeUnit.MILLISECONDS), + "No dispatcher service should be created after the process has been stopped.") + .isInstanceOf(TimeoutException.class); + } + } + @Test void onRemovedExecutionPlan_terminatesRunningJob() throws Exception { executionPlanStore = @@ -550,11 +755,27 @@ void onAddedExecutionPlan_submitsRecoveredJob() throws Exception { final CompletableFuture submittedJobFuture = new CompletableFuture<>(); final TestingDispatcherGateway testingDispatcherGateway = TestingDispatcherGateway.newBuilder() - .setSubmitFunction( - submittedJob -> { - submittedJobFuture.complete(submittedJob); - return CompletableFuture.completedFuture(Acknowledge.get()); + .setRequestApplicationFunction( + applicationId -> + FutureUtils.completedExceptionally( + new FlinkApplicationNotFoundException( + applicationId))) + .setSubmitApplicationFunction( + application -> { + if (application instanceof SingleJobApplication) { + ExecutionPlan submittedJob = + ((SingleJobApplication) application) + .getExecutionPlan(); + submittedJobFuture.complete(submittedJob); + return CompletableFuture.completedFuture(Acknowledge.get()); + } + return FutureUtils.completedExceptionally( + new UnsupportedOperationException()); }) + .setSubmitFunction( + ignored -> + FutureUtils.completedExceptionally( + new UnsupportedOperationException())) .build(); dispatcherServiceFactory = @@ -658,7 +879,7 @@ void recoverJobs_withRecoveryFailure_failsFatally() throws Exception { .setInitialExecutionPlans(Collections.singleton(JOB_GRAPH)) .build(); - runJobRecoveryFailureTest(testException); + runRecoveryFailureTest(testException); } @Test @@ -672,10 +893,39 @@ void recoverJobs_withJobIdRecoveryFailure_failsFatally() throws Exception { }) .build(); - runJobRecoveryFailureTest(testException); + runRecoveryFailureTest(testException); + } + + @Test + void recoverApplications_withRecoveryFailure_failsFatally() throws Exception { + final FlinkException testException = new FlinkException("Test exception"); + applicationStore = + TestingApplicationStore.newBuilder() + .setRecoverApplicationFunction( + (ignoredA, ignoredB) -> { + throw testException; + }) + .setInitialApplications(Collections.singletonList(applicationStoreEntry)) + .build(); + + runRecoveryFailureTest(testException); + } + + @Test + void recoverApplications_withApplicationIdRecoveryFailure_failsFatally() throws Exception { + final FlinkException testException = new FlinkException("Test exception"); + applicationStore = + TestingApplicationStore.newBuilder() + .setApplicationIdsFunction( + ignored -> { + throw testException; + }) + .build(); + + runRecoveryFailureTest(testException); } - private void runJobRecoveryFailureTest(FlinkException testException) throws Exception { + private void runRecoveryFailureTest(FlinkException testException) throws Exception { try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = createDispatcherLeaderProcess()) { dispatcherLeaderProcess.start(); @@ -699,11 +949,20 @@ private void runJobRecoveryFailureTest(FlinkException testException) throws Exce void onAddedExecutionPlan_failingRecoveredJobSubmission_failsFatally() throws Exception { final TestingDispatcherGateway dispatcherGateway = TestingDispatcherGateway.newBuilder() - .setSubmitFunction( - jobGraph -> + .setRequestApplicationFunction( + applicationId -> + FutureUtils.completedExceptionally( + new FlinkApplicationNotFoundException( + applicationId))) + .setSubmitApplicationFunction( + application -> FutureUtils.completedExceptionally( new JobSubmissionException( - jobGraph.getJobID(), "test exception"))) + JOB_GRAPH.getJobID(), "test exception"))) + .setSubmitFunction( + ignored -> + FutureUtils.completedExceptionally( + new UnsupportedOperationException())) .build(); runOnAddedExecutionPlanTest( @@ -724,13 +983,10 @@ private void verifyOnAddedExecutionPlanResultFailsFatally( @Test void onAddedExecutionPlan_duplicateJobSubmissionDueToFalsePositive_willBeIgnored() throws Exception { + // duplicate job submission due to false positive should associate with an application final TestingDispatcherGateway dispatcherGateway = TestingDispatcherGateway.newBuilder() - .setSubmitFunction( - jobGraph -> - FutureUtils.completedExceptionally( - DuplicateJobSubmissionException.of( - jobGraph.getJobID()))) + .setRequestApplicationFunction(applicationId -> new CompletableFuture<>()) .build(); runOnAddedExecutionPlanTest( @@ -776,8 +1032,12 @@ private void runOnAddedExecutionPlanTest( return (ignoredDispatcherId, recoveredJobs, ignoredRecoveredDirtyJobResults, + ignoredRecoveredApplications, + ignoredRecoveredDirtyApplicationResults, ignoredExecutionPlanWriter, - ignoredJobResultStore) -> createFunction.apply(recoveredJobs); + ignoredJobResultStore, + ignoredApplicationStore, + ignoredApplicationResultStore) -> createFunction.apply(recoveredJobs); } private AbstractDispatcherLeaderProcess.DispatcherGatewayServiceFactory @@ -786,8 +1046,12 @@ private void runOnAddedExecutionPlanTest( return (ignoredDispatcherId, ignoredRecoveredJobs, ignoredRecoveredDirtyJobResults, + ignoredRecoveredApplications, + ignoredRecoveredDirtyApplicationResults, ignoredExecutionPlanWriter, - ignoredJobResultStore) -> supplier.get(); + ignoredJobResultStore, + ignoredApplicationStore, + ignoredApplicationResultStore) -> supplier.get(); } private void verifyOnAddedExecutionPlanResultDidNotFail( @@ -804,6 +1068,9 @@ private SessionDispatcherLeaderProcess createDispatcherLeaderProcess() { dispatcherServiceFactory, executionPlanStore, jobResultStore, + applicationStore, + applicationResultStore, + blobServer, ioExecutor, fatalErrorHandler); } 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 87dfa4757956f..3485c9fbba1ea 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 @@ -34,6 +34,8 @@ import org.apache.flink.runtime.dispatcher.SessionDispatcherFactory; import org.apache.flink.runtime.dispatcher.VoidHistoryServerArchivist; import org.apache.flink.runtime.heartbeat.TestingHeartbeatServices; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; +import org.apache.flink.runtime.highavailability.EmbeddedApplicationResultStore; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; @@ -43,8 +45,10 @@ 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.ApplicationStore; import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; -import org.apache.flink.runtime.jobmanager.JobPersistenceComponentFactory; +import org.apache.flink.runtime.jobmanager.PersistenceComponentFactory; +import org.apache.flink.runtime.jobmanager.StandaloneApplicationStore; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.leaderelection.LeaderElection; import org.apache.flink.runtime.leaderelection.TestingLeaderElection; @@ -191,7 +195,7 @@ void testResourceCleanupUnderLeadershipChange() throws Exception { createDispatcherRunner( rpcService, dispatcherLeaderElection, - new JobPersistenceComponentFactory() { + new PersistenceComponentFactory() { @Override public ExecutionPlanStore createExecutionPlanStore() { return createZooKeeperExecutionPlanStore( @@ -202,6 +206,16 @@ public ExecutionPlanStore createExecutionPlanStore() { public JobResultStore createJobResultStore() { return new EmbeddedJobResultStore(); } + + @Override + public ApplicationStore createApplicationStore() { + return new StandaloneApplicationStore(); + } + + @Override + public ApplicationResultStore createApplicationResultStore() { + return new EmbeddedApplicationResultStore(); + } }, partialDispatcherServices, defaultDispatcherRunnerFactory)) { @@ -251,14 +265,14 @@ public JobResultStore createJobResultStore() { private DispatcherRunner createDispatcherRunner( TestingRpcService rpcService, LeaderElection dispatcherLeaderElection, - JobPersistenceComponentFactory jobPersistenceComponentFactory, + PersistenceComponentFactory persistenceComponentFactory, PartialDispatcherServices partialDispatcherServices, DispatcherRunnerFactory dispatcherRunnerFactory) throws Exception { return dispatcherRunnerFactory.createDispatcherRunner( dispatcherLeaderElection, fatalErrorHandler, - jobPersistenceComponentFactory, + persistenceComponentFactory, EXECUTOR_EXTENSION.getExecutor(), 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 80bff9e750069..c5858a97308a0 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.JobPersistenceComponentFactory; +import org.apache.flink.runtime.jobmanager.PersistenceComponentFactory; import org.apache.flink.runtime.leaderelection.LeaderElection; import org.apache.flink.runtime.resourcemanager.ResourceManagerFactory; import org.apache.flink.runtime.resourcemanager.StandaloneResourceManagerFactory; @@ -459,7 +459,7 @@ private TestingDispatcherRunnerFactory( public DispatcherRunner createDispatcherRunner( LeaderElection leaderElection, FatalErrorHandler fatalErrorHandler, - JobPersistenceComponentFactory jobPersistenceComponentFactory, + PersistenceComponentFactory persistenceComponentFactory, Executor ioExecutor, RpcService rpcService, PartialDispatcherServices partialDispatcherServices) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHAApplicationRunITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHAApplicationRunITCase.java new file mode 100644 index 0000000000000..203a0dbfb5640 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/AbstractHAApplicationRunITCase.java @@ -0,0 +1,115 @@ +/* + * 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.ApplicationID; +import org.apache.flink.api.common.ApplicationState; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.testutils.AllCallbackWrapper; +import org.apache.flink.runtime.application.SingleJobApplication; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.JobGraphTestUtils; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.zookeeper.ZooKeeperExtension; +import org.apache.flink.test.junit5.InjectMiniCluster; +import org.apache.flink.testutils.TestingUtils; +import org.apache.flink.testutils.executor.TestExecutorExtension; +import org.apache.flink.util.TestLoggerExtension; +import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; + +import org.junit.jupiter.api.Order; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.time.Duration; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * {@code AbstractApplicationRunITCase} runs a single application in HA mode and provides {@code + * abstract} methods for initializing a specific {@link FileSystem}. + * + *

Sub-classes must use a {@link + * org.apache.flink.runtime.testutils.InternalMiniClusterExtension}. + */ +@ExtendWith(TestLoggerExtension.class) +public abstract class AbstractHAApplicationRunITCase { + + @RegisterExtension + static final TestExecutorExtension EXECUTOR_RESOURCE = + TestingUtils.defaultExecutorExtension(); + + @RegisterExtension + @Order(1) + private static final AllCallbackWrapper ZOOKEEPER_EXTENSION = + new AllCallbackWrapper<>(new ZooKeeperExtension()); + + protected static Configuration addHaConfiguration( + final Configuration config, final String haStoragePath) { + config.set(HighAvailabilityOptions.HA_MODE, "ZOOKEEPER"); + config.set( + HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM, + ZOOKEEPER_EXTENSION.getCustomExtension().getConnectString()); + config.set(HighAvailabilityOptions.HA_STORAGE_PATH, haStoragePath); + + // getFlinkConfiguration() is called on each new instantiation of the MiniCluster which is + // happening before each test run + FileSystem.initialize(config, null); + + return config; + } + + protected void runAfterApplicationTermination() throws Exception {} + + @Test + void testApplicationExecutionInHaMode(@InjectMiniCluster MiniCluster flinkCluster) + throws Exception { + final JobGraph jobGraph = JobGraphTestUtils.singleNoOpJobGraph(); + final SingleJobApplication application = new SingleJobApplication(jobGraph); + final ApplicationID applicationId = application.getApplicationId(); + + // providing a timeout helps making the test fail in case some issue occurred while + // initializing the cluster + flinkCluster.submitApplication(application).get(30, TimeUnit.SECONDS); + + final Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); + final ApplicationState applicationStatus = + FutureUtils.retrySuccessfulWithDelay( + () -> flinkCluster.getApplicationStatus(applicationId), + Duration.ofMillis(10), + deadline, + status -> + flinkCluster.isRunning() + && status == ApplicationState.FINISHED, + new ScheduledExecutorServiceAdapter( + EXECUTOR_RESOURCE.getExecutor())) + .get(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + + assertThat(applicationStatus).isEqualTo(ApplicationState.FINISHED); + + runAfterApplicationTermination(); + } +} 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 4b419a8f25fc9..c9c70f21f38d0 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 @@ -24,6 +24,7 @@ import org.apache.flink.runtime.blob.BlobKey; import org.apache.flink.runtime.blob.BlobStoreService; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; +import org.apache.flink.runtime.jobmanager.ApplicationStore; import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; import org.apache.flink.runtime.testutils.TestingJobResultStore; @@ -218,7 +219,8 @@ private TestingHaServices( throw new AssertionError( "Marking the job as clean shouldn't happen in the HaServices cleanup"); }) - .build()); + .build(), + new EmbeddedApplicationResultStore()); this.closeOperations = closeOperations; this.internalCleanupRunnable = internalCleanupRunnable; this.internalJobCleanupConsumer = internalJobCleanupConsumer; @@ -239,6 +241,11 @@ protected ExecutionPlanStore createExecutionPlanStore() throws Exception { throw new UnsupportedOperationException("Not supported by this test implementation."); } + @Override + protected ApplicationStore createApplicationStore() throws Exception { + 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/ApplicationResultStoreContractTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ApplicationResultStoreContractTest.java new file mode 100644 index 0000000000000..803f69c792073 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/ApplicationResultStoreContractTest.java @@ -0,0 +1,211 @@ +/* + * 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.ApplicationID; +import org.apache.flink.runtime.testutils.TestingApplicationResultStore; + +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.NoSuchElementException; +import java.util.concurrent.CompletionException; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatNoException; +import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; + +/** + * This interface defines a series of tests for any implementation of the {@link + * ApplicationResultStore} to determine whether they correctly implement the contracts defined by + * the interface. + */ +public interface ApplicationResultStoreContractTest { + + ApplicationResultEntry DUMMY_APPLICATION_RESULT_ENTRY = + new ApplicationResultEntry(TestingApplicationResultStore.DUMMY_APPLICATION_RESULT); + + ApplicationResultStore createApplicationResultStore() throws IOException; + + @Test + default void testStoreApplicationResultsWithDuplicateIDsThrowsException() throws IOException { + ApplicationResultStore applicationResultStore = createApplicationResultStore(); + applicationResultStore.createDirtyResultAsync(DUMMY_APPLICATION_RESULT_ENTRY).join(); + final ApplicationResultEntry otherEntryWithDuplicateId = + new ApplicationResultEntry( + TestingApplicationResultStore.createSuccessfulApplicationResult( + DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId())); + assertThatThrownBy( + () -> + applicationResultStore + .createDirtyResultAsync(otherEntryWithDuplicateId) + .join()) + .isInstanceOf(CompletionException.class) + .hasCauseInstanceOf(IllegalStateException.class); + } + + @Test + default void testStoreDirtyEntryForAlreadyCleanedApplicationResultThrowsException() + throws IOException { + ApplicationResultStore applicationResultStore = createApplicationResultStore(); + applicationResultStore.createDirtyResultAsync(DUMMY_APPLICATION_RESULT_ENTRY).join(); + applicationResultStore + .markResultAsCleanAsync(DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()) + .join(); + assertThatThrownBy( + () -> + applicationResultStore + .createDirtyResultAsync(DUMMY_APPLICATION_RESULT_ENTRY) + .join()) + .isInstanceOf(CompletionException.class) + .hasCauseInstanceOf(IllegalStateException.class); + } + + @Test + default void testCleaningDuplicateEntryThrowsNoException() throws IOException { + ApplicationResultStore applicationResultStore = createApplicationResultStore(); + applicationResultStore.createDirtyResultAsync(DUMMY_APPLICATION_RESULT_ENTRY).join(); + applicationResultStore + .markResultAsCleanAsync(DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()) + .join(); + assertThatNoException() + .isThrownBy( + () -> + applicationResultStore + .markResultAsCleanAsync( + DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()) + .join()); + } + + @Test + default void testCleaningNonExistentEntryThrowsException() throws IOException { + ApplicationResultStore applicationResultStore = createApplicationResultStore(); + assertThatThrownBy( + () -> + applicationResultStore + .markResultAsCleanAsync( + DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()) + .join()) + .hasCauseInstanceOf(NoSuchElementException.class); + } + + @Test + default void testHasApplicationResultEntryWithDirtyEntry() throws IOException { + ApplicationResultStore applicationResultStore = createApplicationResultStore(); + applicationResultStore.createDirtyResultAsync(DUMMY_APPLICATION_RESULT_ENTRY).join(); + assertThat( + applicationResultStore + .hasDirtyApplicationResultEntryAsync( + DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()) + .join()) + .isTrue(); + assertThat( + applicationResultStore + .hasCleanApplicationResultEntryAsync( + DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()) + .join()) + .isFalse(); + assertThat( + applicationResultStore + .hasApplicationResultEntryAsync( + DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()) + .join()) + .isTrue(); + } + + @Test + default void testHasApplicationResultEntryWithCleanEntry() throws IOException { + ApplicationResultStore applicationResultStore = createApplicationResultStore(); + applicationResultStore.createDirtyResultAsync(DUMMY_APPLICATION_RESULT_ENTRY).join(); + applicationResultStore + .markResultAsCleanAsync(DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()) + .join(); + assertThat( + applicationResultStore + .hasDirtyApplicationResultEntryAsync( + DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()) + .join()) + .isFalse(); + assertThat( + applicationResultStore + .hasCleanApplicationResultEntryAsync( + DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()) + .join()) + .isTrue(); + assertThat( + applicationResultStore + .hasApplicationResultEntryAsync( + DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()) + .join()) + .isTrue(); + } + + @Test + default void testHasApplicationResultEntryWithEmptyStore() throws IOException { + ApplicationResultStore applicationResultStore = createApplicationResultStore(); + ApplicationID applicationId = new ApplicationID(); + assertThat(applicationResultStore.hasDirtyApplicationResultEntryAsync(applicationId).join()) + .isFalse(); + assertThat(applicationResultStore.hasCleanApplicationResultEntryAsync(applicationId).join()) + .isFalse(); + assertThat(applicationResultStore.hasApplicationResultEntryAsync(applicationId).join()) + .isFalse(); + } + + @Test + default void testGetDirtyResultsWithNoEntry() throws IOException { + ApplicationResultStore applicationResultStore = createApplicationResultStore(); + assertThat(applicationResultStore.getDirtyResults()).isEmpty(); + } + + @Test + default void testGetDirtyResultsWithDirtyEntry() throws IOException { + ApplicationResultStore applicationResultStore = createApplicationResultStore(); + applicationResultStore.createDirtyResultAsync(DUMMY_APPLICATION_RESULT_ENTRY).join(); + assertThat( + applicationResultStore.getDirtyResults().stream() + .map(ApplicationResult::getApplicationId) + .collect(Collectors.toList())) + .singleElement() + .isEqualTo(DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()); + } + + @Test + default void testGetDirtyResultsWithDirtyAndCleanEntry() throws IOException { + ApplicationResultStore applicationResultStore = createApplicationResultStore(); + applicationResultStore.createDirtyResultAsync(DUMMY_APPLICATION_RESULT_ENTRY).join(); + applicationResultStore + .markResultAsCleanAsync(DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()) + .join(); + + final ApplicationResultEntry otherDirtyApplicationResultEntry = + new ApplicationResultEntry( + TestingApplicationResultStore.createSuccessfulApplicationResult( + new ApplicationID())); + applicationResultStore.createDirtyResultAsync(otherDirtyApplicationResultEntry).join(); + + assertThat( + applicationResultStore.getDirtyResults().stream() + .map(ApplicationResult::getApplicationId) + .collect(Collectors.toList())) + .singleElement() + .isEqualTo(otherDirtyApplicationResultEntry.getApplicationId()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemApplicationResultStoreContractTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemApplicationResultStoreContractTest.java new file mode 100644 index 0000000000000..fb4206a089292 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemApplicationResultStoreContractTest.java @@ -0,0 +1,43 @@ +/* + * 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.core.fs.Path; +import org.apache.flink.util.concurrent.Executors; + +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.IOException; + +/** + * Tests for the {@link FileSystemApplicationResultStore} implementation of the {@link + * ApplicationResultStore}'s contracts. + */ +public class FileSystemApplicationResultStoreContractTest + implements ApplicationResultStoreContractTest { + @TempDir File temporaryFolder; + + @Override + public ApplicationResultStore createApplicationResultStore() throws IOException { + Path path = new Path(temporaryFolder.toURI()); + return new FileSystemApplicationResultStore( + path.getFileSystem(), path, false, Executors.directExecutor()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemApplicationResultStoreFileOperationsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemApplicationResultStoreFileOperationsTest.java new file mode 100644 index 0000000000000..dda4673a7d44b --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemApplicationResultStoreFileOperationsTest.java @@ -0,0 +1,346 @@ +/* + * 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.ApplicationID; +import org.apache.flink.core.fs.Path; +import org.apache.flink.core.testutils.FlinkAssertions; +import org.apache.flink.util.FileUtils; +import org.apache.flink.util.TestLoggerExtension; +import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; +import org.apache.flink.util.jackson.JacksonMapperFactory; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +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 org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static org.apache.flink.runtime.highavailability.ApplicationResultStoreContractTest.DUMMY_APPLICATION_RESULT_ENTRY; +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for the internal {@link FileSystemApplicationResultStore} mechanisms. */ +@ExtendWith(TestLoggerExtension.class) +public class FileSystemApplicationResultStoreFileOperationsTest { + + private static final ObjectMapper MAPPER = JacksonMapperFactory.createObjectMapper(); + + private final ManuallyTriggeredScheduledExecutor manuallyTriggeredExecutor = + new ManuallyTriggeredScheduledExecutor(); + + private FileSystemApplicationResultStore fileSystemApplicationResultStore; + + @TempDir File temporaryFolder; + + private Path basePath; + + @BeforeEach + public void setupTest() throws IOException { + basePath = new Path(temporaryFolder.toURI()); + fileSystemApplicationResultStore = + new FileSystemApplicationResultStore( + basePath.getFileSystem(), basePath, false, manuallyTriggeredExecutor); + } + + @Test + public void testValidEntryPathCreation() { + final Path entryParent = + fileSystemApplicationResultStore.constructEntryPath("random-name").getParent(); + assertThat(entryParent) + .extracting( + FileSystemApplicationResultStoreFileOperationsTest::stripSucceedingSlash) + .isEqualTo(stripSucceedingSlash(basePath)); + } + + private static String stripSucceedingSlash(Path path) { + final String uriStr = path.toUri().toString(); + if (uriStr.charAt(uriStr.length() - 1) == '/') { + return uriStr.substring(0, uriStr.length() - 1); + } + + return uriStr; + } + + @Test + public void testHasValidApplicationResultStoreEntryExtension() { + assertThat( + FileSystemApplicationResultStore + .hasValidApplicationResultStoreEntryExtension( + "test" + FileSystemApplicationResultStore.FILE_EXTENSION)) + .isTrue(); + } + + @ParameterizedTest + @ValueSource(strings = {"test.txt", "", "test.zip"}) + public void testHasInvalidApplicationResultStoreEntryExtension(String filename) { + assertThat( + FileSystemApplicationResultStore + .hasValidApplicationResultStoreEntryExtension(filename)) + .isFalse(); + } + + @Test + public void testHasValidDirtyApplicationResultStoreEntryExtension() { + assertThat( + FileSystemApplicationResultStore + .hasValidDirtyApplicationResultStoreEntryExtension( + "test" + + FileSystemApplicationResultStore + .DIRTY_FILE_EXTENSION)) + .isTrue(); + } + + @ParameterizedTest + @ValueSource(strings = {"test.json", "test.txt", "", "test.zip"}) + public void testHasInvalidDirtyApplicationResultStoreEntryExtension(String filename) { + assertThat( + FileSystemApplicationResultStore + .hasValidDirtyApplicationResultStoreEntryExtension(filename)) + .isFalse(); + } + + @Test + public void testBaseDirectoryCreationOnResultStoreInitialization() throws Exception { + final File emptyBaseDirectory = new File(temporaryFolder.getPath(), "empty-temp-dir"); + final Path basePath = new Path(emptyBaseDirectory.getPath()); + assertThat(emptyBaseDirectory).doesNotExist(); + + fileSystemApplicationResultStore = + new FileSystemApplicationResultStore( + basePath.getFileSystem(), basePath, false, manuallyTriggeredExecutor); + // Result store operations are creating the base directory on-the-fly + assertThat(emptyBaseDirectory).doesNotExist(); + CompletableFuture dirtyResultAsync = + fileSystemApplicationResultStore.createDirtyResultAsync( + DUMMY_APPLICATION_RESULT_ENTRY); + assertThat(emptyBaseDirectory).doesNotExist(); + manuallyTriggeredExecutor.triggerAll(); + FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds(); + assertThat(emptyBaseDirectory).exists().isDirectory(); + } + + @Test + public void testStoreDirtyApplicationResultCreatesFile() throws Exception { + CompletableFuture dirtyResultAsync = + fileSystemApplicationResultStore.createDirtyResultAsync( + DUMMY_APPLICATION_RESULT_ENTRY); + assertThat(expectedDirtyFile(DUMMY_APPLICATION_RESULT_ENTRY)).doesNotExist(); + manuallyTriggeredExecutor.triggerAll(); + FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds(); + assertThat(getCleanResultIdsFromFileSystem()).isEmpty(); + assertThat(expectedDirtyFile(DUMMY_APPLICATION_RESULT_ENTRY)) + .exists() + .isFile() + .isNotEmpty(); + } + + @Test + public void testStoreCleanApplicationResultCreatesFile() throws Exception { + CompletableFuture dirtyResultAsync = + fileSystemApplicationResultStore.createDirtyResultAsync( + DUMMY_APPLICATION_RESULT_ENTRY); + manuallyTriggeredExecutor.triggerAll(); + FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds(); + CompletableFuture markCleanAsync = + fileSystemApplicationResultStore.markResultAsCleanAsync( + DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()); + assertThat(getCleanResultIdsFromFileSystem()) + .doesNotContain(DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()); + manuallyTriggeredExecutor.triggerAll(); + FlinkAssertions.assertThatFuture(markCleanAsync).eventuallySucceeds(); + assertThat(getCleanResultIdsFromFileSystem()) + .containsExactlyInAnyOrder(DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()); + } + + @Test + public void testStoreCleanApplicationResultDeletesDirtyFile() { + CompletableFuture dirtyResultAsync = + fileSystemApplicationResultStore.createDirtyResultAsync( + DUMMY_APPLICATION_RESULT_ENTRY); + assertThat(expectedDirtyFile(DUMMY_APPLICATION_RESULT_ENTRY)).doesNotExist(); + manuallyTriggeredExecutor.triggerAll(); + FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds(); + assertThat(expectedDirtyFile(DUMMY_APPLICATION_RESULT_ENTRY)) + .exists() + .isFile() + .isNotEmpty(); + + CompletableFuture markResultAsCleanAsync = + fileSystemApplicationResultStore.markResultAsCleanAsync( + DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()); + manuallyTriggeredExecutor.triggerAll(); + FlinkAssertions.assertThatFuture(markResultAsCleanAsync).eventuallySucceeds(); + assertThat(expectedDirtyFile(DUMMY_APPLICATION_RESULT_ENTRY)).doesNotExist(); + } + + @Test + public void testCleanDirtyApplicationResultTwiceIsIdempotent() throws IOException { + CompletableFuture dirtyResultAsync = + fileSystemApplicationResultStore.createDirtyResultAsync( + DUMMY_APPLICATION_RESULT_ENTRY); + manuallyTriggeredExecutor.triggerAll(); + FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds(); + CompletableFuture cleanResultAsync = + fileSystemApplicationResultStore.markResultAsCleanAsync( + DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()); + manuallyTriggeredExecutor.triggerAll(); + FlinkAssertions.assertThatFuture(cleanResultAsync).eventuallySucceeds(); + final byte[] cleanFileData = + FileUtils.readAllBytes(expectedCleanFile(DUMMY_APPLICATION_RESULT_ENTRY).toPath()); + + CompletableFuture markResultAsCleanAsync = + fileSystemApplicationResultStore.markResultAsCleanAsync( + DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()); + manuallyTriggeredExecutor.triggerAll(); + FlinkAssertions.assertThatFuture(markResultAsCleanAsync).eventuallySucceeds(); + assertThat(expectedCleanFile(DUMMY_APPLICATION_RESULT_ENTRY)) + .as( + "Marking the same application %s as clean should be idempotent.", + DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()) + .hasBinaryContent(cleanFileData); + } + + /** + * Tests that, when the application result store is configured to delete on commit, both the + * clean and the dirty files for an application entry are deleted when the result is marked as + * clean. + */ + @Test + public void testDeleteOnCommit() throws IOException { + Path path = new Path(temporaryFolder.toURI()); + fileSystemApplicationResultStore = + new FileSystemApplicationResultStore( + path.getFileSystem(), path, true, manuallyTriggeredExecutor); + + CompletableFuture dirtyResultAsync = + fileSystemApplicationResultStore.createDirtyResultAsync( + DUMMY_APPLICATION_RESULT_ENTRY); + assertThat(expectedDirtyFile(DUMMY_APPLICATION_RESULT_ENTRY)).doesNotExist(); + manuallyTriggeredExecutor.triggerAll(); + FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds(); + assertThat(expectedDirtyFile(DUMMY_APPLICATION_RESULT_ENTRY)) + .exists() + .isFile() + .isNotEmpty(); + + CompletableFuture markResultAsCleanAsync = + fileSystemApplicationResultStore.markResultAsCleanAsync( + DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()); + manuallyTriggeredExecutor.triggerAll(); + FlinkAssertions.assertThatFuture(markResultAsCleanAsync).eventuallySucceeds(); + assertThat(expectedDirtyFile(DUMMY_APPLICATION_RESULT_ENTRY)).doesNotExist(); + assertThat(expectedCleanFile(DUMMY_APPLICATION_RESULT_ENTRY)).doesNotExist(); + } + + @Test + public void testVersionSerialization() throws IOException { + CompletableFuture dirtyResultAsync = + fileSystemApplicationResultStore.createDirtyResultAsync( + DUMMY_APPLICATION_RESULT_ENTRY); + manuallyTriggeredExecutor.triggerAll(); + FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds(); + final File dirtyFile = expectedDirtyFile(DUMMY_APPLICATION_RESULT_ENTRY); + final FileSystemApplicationResultStore.JsonApplicationResultEntry deserializedEntry = + MAPPER.readValue( + dirtyFile, + FileSystemApplicationResultStore.JsonApplicationResultEntry.class); + assertThat(dirtyFile).isFile().content().containsPattern("\"version\":1"); + assertThat(deserializedEntry.getVersion()).isEqualTo(1); + } + + @Test + public void testApplicationResultSerializationDeserialization() throws IOException { + CompletableFuture dirtyResultAsync = + fileSystemApplicationResultStore.createDirtyResultAsync( + DUMMY_APPLICATION_RESULT_ENTRY); + manuallyTriggeredExecutor.triggerAll(); + FlinkAssertions.assertThatFuture(dirtyResultAsync).eventuallySucceeds(); + final File dirtyFile = expectedDirtyFile(DUMMY_APPLICATION_RESULT_ENTRY); + final FileSystemApplicationResultStore.JsonApplicationResultEntry deserializedEntry = + MAPPER.readValue( + dirtyFile, + FileSystemApplicationResultStore.JsonApplicationResultEntry.class); + final ApplicationResult deserializedApplicationResult = + deserializedEntry.getApplicationResult(); + assertThat(deserializedApplicationResult) + .extracting(ApplicationResult::getApplicationId) + .isEqualTo(DUMMY_APPLICATION_RESULT_ENTRY.getApplicationId()); + assertThat(deserializedApplicationResult) + .extracting(ApplicationResult::getApplicationState) + .isEqualTo( + DUMMY_APPLICATION_RESULT_ENTRY + .getApplicationResult() + .getApplicationState()); + } + + private List getCleanResultIdsFromFileSystem() throws IOException { + final List cleanResults = new ArrayList<>(); + + final File[] cleanFiles = + temporaryFolder.listFiles( + (dir, name) -> + !FileSystemApplicationResultStore + .hasValidDirtyApplicationResultStoreEntryExtension(name)); + assert cleanFiles != null; + for (File cleanFile : cleanFiles) { + final FileSystemApplicationResultStore.JsonApplicationResultEntry entry = + MAPPER.readValue( + cleanFile, + FileSystemApplicationResultStore.JsonApplicationResultEntry.class); + cleanResults.add(entry.getApplicationResult().getApplicationId()); + } + + return cleanResults; + } + + /** + * Generates the expected path for a dirty entry given an application entry. + * + * @param entry The application ID to construct the expected dirty path from. + * @return The expected dirty file. + */ + private File expectedDirtyFile(ApplicationResultEntry entry) { + return new File( + temporaryFolder.toURI().getPath(), + entry.getApplicationId().toString() + + FileSystemApplicationResultStore.DIRTY_FILE_EXTENSION); + } + + /** + * Generates the expected path for a clean entry given an application entry. + * + * @param entry The application entry to construct the expected clean path from. + * @return The expected clean file. + */ + private File expectedCleanFile(ApplicationResultEntry entry) { + return new File( + temporaryFolder.toURI().getPath(), + entry.getApplicationId().toString() + + FileSystemApplicationResultStore.FILE_EXTENSION); + } +} 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 38e9ed1b3b43f..2885f45b00fea 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 @@ -23,6 +23,7 @@ import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedJobResultStore; +import org.apache.flink.runtime.jobmanager.ApplicationStore; import org.apache.flink.runtime.jobmanager.ExecutionPlanStore; import org.apache.flink.runtime.leaderelection.LeaderElection; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService; @@ -70,6 +71,11 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices private volatile JobResultStore jobResultStore = new EmbeddedJobResultStore(); + private volatile ApplicationStore applicationStore; + + private volatile ApplicationResultStore applicationResultStore = + new EmbeddedApplicationResultStore(); + private CompletableFuture closeFuture = new CompletableFuture<>(); private CompletableFuture cleanupAllDataFuture = new CompletableFuture<>(); @@ -128,6 +134,14 @@ public void setJobResultStore(JobResultStore jobResultStore) { this.jobResultStore = jobResultStore; } + public void setApplicationStore(ApplicationStore applicationStore) { + this.applicationStore = applicationStore; + } + + public void setApplicationResultStore(ApplicationResultStore applicationResultStore) { + this.applicationResultStore = applicationResultStore; + } + public void setJobMasterLeaderElectionFunction( Function jobMasterLeaderElectionServiceFunction) { this.jobMasterLeaderElectionServiceFunction = jobMasterLeaderElectionServiceFunction; @@ -264,6 +278,22 @@ public JobResultStore getJobResultStore() { return jobResultStore; } + @Override + public ApplicationStore getApplicationStore() { + ApplicationStore store = applicationStore; + + if (store != null) { + return store; + } else { + throw new IllegalStateException("ApplicationStore has not been set"); + } + } + + @Override + public ApplicationResultStore getApplicationResultStore() { + return applicationResultStore; + } + @Override public BlobStore createBlobStore() throws IOException { return new VoidBlobStore(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedApplicationResultStoreContractTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedApplicationResultStoreContractTest.java new file mode 100644 index 0000000000000..893f5c87fba6f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedApplicationResultStoreContractTest.java @@ -0,0 +1,37 @@ +/* + * 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.runtime.highavailability.ApplicationResultStore; +import org.apache.flink.runtime.highavailability.ApplicationResultStoreContractTest; +import org.apache.flink.runtime.highavailability.EmbeddedApplicationResultStore; + +import java.io.IOException; + +/** + * Tests for the {@link EmbeddedApplicationResultStore} implementation of the {@link + * ApplicationResultStore}'s contracts. + */ +public class EmbeddedApplicationResultStoreContractTest + implements ApplicationResultStoreContractTest { + @Override + public ApplicationResultStore createApplicationResultStore() throws IOException { + return new EmbeddedApplicationResultStore(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultApplicationStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultApplicationStoreTest.java new file mode 100644 index 0000000000000..e37b4ca91abfb --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultApplicationStoreTest.java @@ -0,0 +1,279 @@ +/* + * 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.api.common.ApplicationID; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.runtime.checkpoint.TestingRetrievableStateStorageHelper; +import org.apache.flink.runtime.persistence.IntegerResourceVersion; +import org.apache.flink.runtime.persistence.StateHandleStore; +import org.apache.flink.runtime.persistence.TestingStateHandleStore; +import org.apache.flink.runtime.state.RetrievableStateHandle; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.concurrent.Executors; + +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * Tests for {@link DefaultApplicationStore} with {@link TestingStateHandleStore} and {@link + * TestingApplicationStoreEntry}. + */ +public class DefaultApplicationStoreTest extends TestLogger { + + private final ApplicationStoreEntry testingApplication = + TestingApplicationStoreEntry.newBuilder().build(); + private final long timeout = 100L; + + private TestingStateHandleStore.Builder builder; + private TestingRetrievableStateStorageHelper storageHelper; + + @Before + public void setup() { + builder = TestingStateHandleStore.newBuilder(); + storageHelper = new TestingRetrievableStateStorageHelper<>(); + } + + @Test + public void testRecoverApplication() throws Exception { + final RetrievableStateHandle stateHandle = + storageHelper.store(testingApplication); + final TestingStateHandleStore stateHandleStore = + builder.setGetFunction(ignore -> stateHandle).build(); + + final ApplicationStore applicationStore = createAndStartApplicationStore(stateHandleStore); + + final Optional recoveredApplication = + applicationStore.recoverApplication(testingApplication.getApplicationId()); + assertThat(recoveredApplication).isPresent(); + assertThat(recoveredApplication.get().getApplicationId()) + .isEqualTo(testingApplication.getApplicationId()); + } + + @Test + public void testRecoverApplicationWhenNotExist() throws Exception { + final TestingStateHandleStore stateHandleStore = + builder.setGetFunction( + ignore -> { + throw new StateHandleStore.NotExistException("Not exist."); + }) + .build(); + + final ApplicationStore applicationStore = createAndStartApplicationStore(stateHandleStore); + + final Optional recoveredApplication = + applicationStore.recoverApplication(testingApplication.getApplicationId()); + assertThat(recoveredApplication).isEmpty(); + } + + @Test + public void testRecoverApplicationFailedShouldReleaseHandle() throws Exception { + final CompletableFuture releaseFuture = new CompletableFuture<>(); + final FlinkException testException = new FlinkException("Test exception."); + final TestingStateHandleStore stateHandleStore = + builder.setGetFunction( + ignore -> { + throw testException; + }) + .setReleaseConsumer(releaseFuture::complete) + .build(); + + final ApplicationStore applicationStore = createAndStartApplicationStore(stateHandleStore); + + assertThatThrownBy( + () -> + applicationStore.recoverApplication( + testingApplication.getApplicationId())) + .hasCause(testException); + String actual = releaseFuture.get(timeout, TimeUnit.MILLISECONDS); + assertThat(testingApplication.getApplicationId()).hasToString(actual); + } + + @Test + public void testPutApplicationWhenNotExist() throws Exception { + final CompletableFuture addFuture = new CompletableFuture<>(); + final TestingStateHandleStore stateHandleStore = + builder.setExistsFunction(ignore -> IntegerResourceVersion.notExisting()) + .setAddFunction( + (ignore, state) -> { + addFuture.complete(state); + return storageHelper.store(state); + }) + .build(); + + final ApplicationStore applicationStore = createAndStartApplicationStore(stateHandleStore); + applicationStore.putApplication(testingApplication); + + final ApplicationStoreEntry actual = addFuture.get(timeout, TimeUnit.MILLISECONDS); + assertThat(actual.getApplicationId()).isEqualTo(testingApplication.getApplicationId()); + } + + @Test + public void testPutApplicationWhenAlreadyExist() throws Exception { + final CompletableFuture> + replaceFuture = new CompletableFuture<>(); + final int resourceVersion = 100; + final AtomicBoolean alreadyExist = new AtomicBoolean(false); + final TestingStateHandleStore stateHandleStore = + builder.setExistsFunction( + ignore -> { + if (alreadyExist.get()) { + return IntegerResourceVersion.valueOf(resourceVersion); + } else { + alreadyExist.set(true); + return IntegerResourceVersion.notExisting(); + } + }) + .setAddFunction((ignore, state) -> storageHelper.store(state)) + .setReplaceConsumer(replaceFuture::complete) + .build(); + + final ApplicationStore applicationStore = createAndStartApplicationStore(stateHandleStore); + applicationStore.putApplication(testingApplication); + // Replace + applicationStore.putApplication(testingApplication); + + final Tuple3 actual = + replaceFuture.get(timeout, TimeUnit.MILLISECONDS); + assertThat(actual.f0).isEqualTo(testingApplication.getApplicationId().toString()); + assertThat(actual.f1).isEqualTo(IntegerResourceVersion.valueOf(resourceVersion)); + assertThat(actual.f2.getApplicationId()).isEqualTo(testingApplication.getApplicationId()); + } + + @Test + public void testGlobalCleanup() throws Exception { + final CompletableFuture removeFuture = new CompletableFuture<>(); + final TestingStateHandleStore stateHandleStore = + builder.setAddFunction((ignore, state) -> storageHelper.store(state)) + .setRemoveFunction( + name -> { + removeFuture.complete(ApplicationID.fromHexString(name)); + return true; + }) + .build(); + + final ApplicationStore applicationStore = createAndStartApplicationStore(stateHandleStore); + + applicationStore.putApplication(testingApplication); + applicationStore + .globalCleanupAsync( + testingApplication.getApplicationId(), Executors.directExecutor()) + .join(); + final ApplicationID actual = removeFuture.get(timeout, TimeUnit.MILLISECONDS); + assertThat(actual).isEqualTo(testingApplication.getApplicationId()); + } + + @Test + public void testGlobalCleanupWithNonExistName() throws Exception { + final CompletableFuture removeFuture = new CompletableFuture<>(); + final TestingStateHandleStore stateHandleStore = + builder.setRemoveFunction( + name -> { + removeFuture.complete(ApplicationID.fromHexString(name)); + return true; + }) + .build(); + + final ApplicationStore applicationStore = createAndStartApplicationStore(stateHandleStore); + applicationStore + .globalCleanupAsync( + testingApplication.getApplicationId(), Executors.directExecutor()) + .join(); + + assertThat(removeFuture).isDone(); + } + + @Test + public void testGlobalCleanupFailsIfRemovalReturnsFalse() throws Exception { + final TestingStateHandleStore stateHandleStore = + builder.setRemoveFunction(name -> false).build(); + + final ApplicationStore applicationStore = createAndStartApplicationStore(stateHandleStore); + assertThatThrownBy( + () -> + applicationStore + .globalCleanupAsync( + testingApplication.getApplicationId(), + Executors.directExecutor()) + .get()) + .isInstanceOf(ExecutionException.class); + } + + @Test + public void testGetApplicationIds() throws Exception { + final Collection existingApplicationIds = + Arrays.asList(new ApplicationID(), new ApplicationID()); + final TestingStateHandleStore stateHandleStore = + builder.setGetAllHandlesSupplier( + () -> + existingApplicationIds.stream() + .map(ApplicationID::toString) + .collect(Collectors.toList())) + .build(); + + final ApplicationStore applicationStore = createAndStartApplicationStore(stateHandleStore); + final Collection applicationIds = applicationStore.getApplicationIds(); + assertThat(applicationIds).containsExactlyInAnyOrderElementsOf(existingApplicationIds); + } + + @Test + public void testStoppingApplicationStoreShouldReleaseAllHandles() throws Exception { + final CompletableFuture completableFuture = new CompletableFuture<>(); + final TestingStateHandleStore stateHandleStore = + builder.setReleaseAllHandlesRunnable(() -> completableFuture.complete(null)) + .build(); + final ApplicationStore applicationStore = createAndStartApplicationStore(stateHandleStore); + applicationStore.stop(); + + assertThat(completableFuture).isDone(); + } + + private ApplicationStore createAndStartApplicationStore( + TestingStateHandleStore stateHandleStore) throws Exception { + final ApplicationStore applicationStore = + new DefaultApplicationStore<>( + stateHandleStore, + new ApplicationStoreUtil() { + @Override + public String applicationIdToName(ApplicationID applicationId) { + return applicationId.toString(); + } + + @Override + public ApplicationID nameToApplicationId(String name) { + return ApplicationID.fromHexString(name); + } + }); + applicationStore.start(); + return applicationStore; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneApplicationStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneApplicationStoreTest.java new file mode 100644 index 0000000000000..984554db8e986 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/StandaloneApplicationStoreTest.java @@ -0,0 +1,54 @@ +/* + * 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.api.common.ApplicationID; +import org.apache.flink.util.concurrent.Executors; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +/** Tests for the {@link StandaloneApplicationStore}. */ +public class StandaloneApplicationStoreTest { + + /** Tests that all operations work and don't change the state. */ + @Test + public void testNoOps() throws Exception { + StandaloneApplicationStore applicationStore = new StandaloneApplicationStore(); + applicationStore.start(); + + ApplicationID applicationId = new ApplicationID(); + ApplicationStoreEntry applicationStoreEntry = + TestingApplicationStoreEntry.newBuilder().setApplicationId(applicationId).build(); + + assertEquals(0, applicationStore.getApplicationIds().size()); + + applicationStore.putApplication(applicationStoreEntry); + assertEquals(0, applicationStore.getApplicationIds().size()); + + applicationStore.globalCleanupAsync(applicationId, Executors.directExecutor()).join(); + assertEquals(0, applicationStore.getApplicationIds().size()); + + assertFalse(applicationStore.recoverApplication(applicationId).isPresent()); + + applicationStore.stop(); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/TestingApplicationStoreEntry.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/TestingApplicationStoreEntry.java new file mode 100644 index 0000000000000..f303365f26f1f --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/TestingApplicationStoreEntry.java @@ -0,0 +1,102 @@ +/* + * 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.api.common.ApplicationID; +import org.apache.flink.api.common.JobInfo; +import org.apache.flink.runtime.application.AbstractApplication; +import org.apache.flink.runtime.blob.PermanentBlobService; + +import javax.annotation.Nullable; + +import java.util.Collection; + +/** {@link ApplicationStoreEntry} implementation for testing purposes. */ +public class TestingApplicationStoreEntry implements ApplicationStoreEntry { + + private final ApplicationID applicationId; + private final String name; + @Nullable private final AbstractApplication application; + + public TestingApplicationStoreEntry(ApplicationID applicationId, String name) { + this.applicationId = applicationId; + this.name = name; + this.application = null; + } + + public TestingApplicationStoreEntry(AbstractApplication application) { + this.applicationId = application.getApplicationId(); + this.name = application.getName(); + this.application = application; + } + + @Override + public AbstractApplication getApplication( + PermanentBlobService blobService, + Collection recoveredJobInfos, + Collection recoveredTerminalJobInfos) { + if (application != null) { + return application; + } + throw new UnsupportedOperationException(); + } + + @Override + public ApplicationID getApplicationId() { + return applicationId; + } + + @Override + public String getName() { + return name; + } + + public static Builder newBuilder() { + return new Builder(); + } + + /** Builder for creating {@link TestingApplicationStoreEntry} instances. */ + public static class Builder { + private ApplicationID applicationId = new ApplicationID(); + private String name = "TestingApplication"; + private AbstractApplication application; + + public Builder setApplicationId(ApplicationID applicationId) { + this.applicationId = applicationId; + return this; + } + + public Builder setName(String name) { + this.name = name; + return this; + } + + public Builder setApplication(AbstractApplication application) { + this.application = application; + return this; + } + + public TestingApplicationStoreEntry build() { + if (application != null) { + return new TestingApplicationStoreEntry(application); + } + return new TestingApplicationStoreEntry(applicationId, name); + } + } +} 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/TestingPersistenceComponentFactory.java similarity index 58% rename from flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/TestingJobPersistenceComponentFactory.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/TestingPersistenceComponentFactory.java index 8a8435a109484..03bc5c19bd6ea 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/TestingJobPersistenceComponentFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/TestingPersistenceComponentFactory.java @@ -18,22 +18,30 @@ package org.apache.flink.runtime.jobmanager; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; import org.apache.flink.runtime.highavailability.JobResultStore; import org.apache.flink.util.Preconditions; /** - * {@code TestingJobPersistenceComponentFactory} implements {@link JobPersistenceComponentFactory} - * for a given {@link ExecutionPlanStore} and {@link JobResultStore}. + * {@code TestingPersistenceComponentFactory} implements {@link PersistenceComponentFactory} for a + * given {@link ExecutionPlanStore} and {@link JobResultStore}. */ -public class TestingJobPersistenceComponentFactory implements JobPersistenceComponentFactory { +public class TestingPersistenceComponentFactory implements PersistenceComponentFactory { private final ExecutionPlanStore executionPlanStore; private final JobResultStore jobResultStore; + private final ApplicationStore applicationStore; + private final ApplicationResultStore applicationResultStore; - public TestingJobPersistenceComponentFactory( - ExecutionPlanStore executionPlanStore, JobResultStore jobResultStore) { + public TestingPersistenceComponentFactory( + ExecutionPlanStore executionPlanStore, + JobResultStore jobResultStore, + ApplicationStore applicationStore, + ApplicationResultStore applicationResultStore) { this.executionPlanStore = Preconditions.checkNotNull(executionPlanStore); this.jobResultStore = Preconditions.checkNotNull(jobResultStore); + this.applicationStore = applicationStore; + this.applicationResultStore = applicationResultStore; } @Override @@ -45,4 +53,14 @@ public ExecutionPlanStore createExecutionPlanStore() { public JobResultStore createJobResultStore() { return jobResultStore; } + + @Override + public ApplicationStore createApplicationStore() { + return applicationStore; + } + + @Override + public ApplicationResultStore createApplicationResultStore() { + return applicationResultStore; + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperApplicationStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperApplicationStoreITCase.java new file mode 100644 index 0000000000000..2cd4b89bb3b8b --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperApplicationStoreITCase.java @@ -0,0 +1,297 @@ +/* + * 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.api.common.ApplicationID; +import org.apache.flink.core.testutils.EachCallbackWrapper; +import org.apache.flink.runtime.persistence.RetrievableStateStorageHelper; +import org.apache.flink.runtime.state.RetrievableStreamStateHandle; +import org.apache.flink.runtime.state.memory.ByteStreamStateHandle; +import org.apache.flink.runtime.util.TestingFatalErrorHandlerExtension; +import org.apache.flink.runtime.zookeeper.ZooKeeperExtension; +import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.concurrent.Executors; + +import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import javax.annotation.Nonnull; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Optional; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + +/** + * IT tests for {@link DefaultApplicationStore} with all ZooKeeper components(e.g. {@link + * ZooKeeperStateHandleStore}, {@link ZooKeeperApplicationStoreUtil}). + */ +public class ZooKeeperApplicationStoreITCase extends TestLogger { + + private final ZooKeeperExtension zooKeeperExtension = new ZooKeeperExtension(); + + @RegisterExtension + final EachCallbackWrapper zooKeeperResource = + new EachCallbackWrapper<>(zooKeeperExtension); + + @RegisterExtension + final TestingFatalErrorHandlerExtension testingFatalErrorHandlerResource = + new TestingFatalErrorHandlerExtension(); + + private static final RetrievableStateStorageHelper localStateStorage = + applicationStoreEntry -> { + ByteStreamStateHandle byteStreamStateHandle = + new ByteStreamStateHandle( + String.valueOf(java.util.UUID.randomUUID()), + InstantiationUtil.serializeObject(applicationStoreEntry)); + return new RetrievableStreamStateHandle<>(byteStreamStateHandle); + }; + + @Test + public void testPutAndRemoveApplication() throws Exception { + ApplicationStore applicationStore = + createZooKeeperApplicationStore("/testPutAndRemoveApplication"); + + try { + applicationStore.start(); + + ApplicationStoreEntry applicationEntry = + createApplicationStoreEntry(new ApplicationID(), "AppName"); + + // Empty state + assertThat(applicationStore.getApplicationIds()).isEmpty(); + + // Add initial + applicationStore.putApplication(applicationEntry); + + // Verify initial application + Collection applicationIds = applicationStore.getApplicationIds(); + assertThat(applicationIds).hasSize(1); + + ApplicationID applicationId = applicationIds.iterator().next(); + + Optional recoveredEntry = + applicationStore.recoverApplication(applicationId); + assertThat(recoveredEntry).isPresent(); + verifyApplicationStoreEntries(applicationEntry, recoveredEntry.get()); + + // Update (same ID) + applicationEntry = + createApplicationStoreEntry( + applicationEntry.getApplicationId(), "Updated AppName"); + applicationStore.putApplication(applicationEntry); + + // Verify updated + applicationIds = applicationStore.getApplicationIds(); + assertThat(applicationIds).hasSize(1); + + applicationId = applicationIds.iterator().next(); + + recoveredEntry = applicationStore.recoverApplication(applicationId); + assertThat(recoveredEntry).isPresent(); + verifyApplicationStoreEntries(applicationEntry, recoveredEntry.get()); + + // Remove + applicationStore + .globalCleanupAsync( + applicationEntry.getApplicationId(), Executors.directExecutor()) + .join(); + + // Empty state + assertThat(applicationStore.getApplicationIds()).isEmpty(); + + // Don't fail if called again + applicationStore + .globalCleanupAsync( + applicationEntry.getApplicationId(), Executors.directExecutor()) + .join(); + } finally { + applicationStore.stop(); + } + } + + @Nonnull + private ApplicationStore createZooKeeperApplicationStore(String fullPath) throws Exception { + final CuratorFramework client = + zooKeeperExtension.getZooKeeperClient( + testingFatalErrorHandlerResource.getTestingFatalErrorHandler()); + // Ensure that the applications path exists + client.newNamespaceAwareEnsurePath(fullPath).ensure(client.getZookeeperClient()); + + // All operations will have the path as root + CuratorFramework facade = client.usingNamespace(client.getNamespace() + fullPath); + final ZooKeeperStateHandleStore zooKeeperStateHandleStore = + new ZooKeeperStateHandleStore<>(facade, localStateStorage); + return new DefaultApplicationStore<>( + zooKeeperStateHandleStore, ZooKeeperApplicationStoreUtil.INSTANCE); + } + + @Test + public void testRecoverApplications() throws Exception { + ApplicationStore applicationStore = + createZooKeeperApplicationStore("/testRecoverApplications"); + + try { + applicationStore.start(); + + HashMap expected = new HashMap<>(); + ApplicationID[] applicationIds = + new ApplicationID[] { + new ApplicationID(), new ApplicationID(), new ApplicationID() + }; + + expected.put(applicationIds[0], createApplicationStoreEntry(applicationIds[0])); + expected.put(applicationIds[1], createApplicationStoreEntry(applicationIds[1])); + expected.put(applicationIds[2], createApplicationStoreEntry(applicationIds[2])); + + // Add all + for (ApplicationStoreEntry applicationStoreEntry : expected.values()) { + applicationStore.putApplication(applicationStoreEntry); + } + + Collection actual = applicationStore.getApplicationIds(); + + assertThat(actual).hasSameSizeAs(expected.entrySet()); + + for (ApplicationID applicationId : actual) { + Optional applicationStoreEntry = + applicationStore.recoverApplication(applicationId); + assertThat(applicationStoreEntry).isPresent(); + assertThat(expected).containsKey(applicationStoreEntry.get().getApplicationId()); + + verifyApplicationStoreEntries( + expected.get(applicationStoreEntry.get().getApplicationId()), + applicationStoreEntry.get()); + + applicationStore + .globalCleanupAsync( + applicationStoreEntry.get().getApplicationId(), + Executors.directExecutor()) + .join(); + } + + // Empty state + assertThat(applicationStore.getApplicationIds()).isEmpty(); + } finally { + applicationStore.stop(); + } + } + + @Test + public void testUpdateApplicationYouDidNotGetOrAdd() throws Exception { + ApplicationStore applicationStore = + createZooKeeperApplicationStore("/testUpdateApplicationYouDidNotGetOrAdd"); + + ApplicationStore otherApplicationStore = + createZooKeeperApplicationStore("/testUpdateApplicationYouDidNotGetOrAdd"); + + applicationStore.start(); + otherApplicationStore.start(); + + ApplicationStoreEntry applicationEntry = createApplicationStoreEntry(new ApplicationID()); + + applicationStore.putApplication(applicationEntry); + + assertThatExceptionOfType(IllegalStateException.class) + .isThrownBy(() -> otherApplicationStore.putApplication(applicationEntry)); + + applicationStore.stop(); + otherApplicationStore.stop(); + } + + /** + * Tests that we fail with an exception if the application cannot be removed from the + * ZooKeeperApplicationStore. + * + *

Tests that a close ZooKeeperApplicationStore no longer holds any locks. + */ + @Test + public void testApplicationRemovalFailureAndLockRelease() throws Exception { + final ApplicationStore submittedApplicationStore = + createZooKeeperApplicationStore("/testApplicationRemovalFailureAndLockRelease"); + final ApplicationStore otherSubmittedApplicationStore = + createZooKeeperApplicationStore("/testApplicationRemovalFailureAndLockRelease"); + + submittedApplicationStore.start(); + otherSubmittedApplicationStore.start(); + + final ApplicationStoreEntry applicationEntry = + createApplicationStoreEntry(new ApplicationID()); + submittedApplicationStore.putApplication(applicationEntry); + + final Optional recoveredApplicationEntry = + otherSubmittedApplicationStore.recoverApplication( + applicationEntry.getApplicationId()); + + assertThat(recoveredApplicationEntry).isPresent(); + + assertThatExceptionOfType(Exception.class) + .as( + "It should not be possible to remove the ApplicationStoreEntry since the first store still has a lock on it.") + .isThrownBy( + () -> + otherSubmittedApplicationStore + .globalCleanupAsync( + recoveredApplicationEntry.get().getApplicationId(), + Executors.directExecutor()) + .join()); + + submittedApplicationStore.stop(); + + // now we should be able to delete the application entry + otherSubmittedApplicationStore + .globalCleanupAsync( + recoveredApplicationEntry.get().getApplicationId(), + Executors.directExecutor()) + .join(); + + assertThat( + otherSubmittedApplicationStore.recoverApplication( + recoveredApplicationEntry.get().getApplicationId())) + .isEmpty(); + + otherSubmittedApplicationStore.stop(); + } + + // --------------------------------------------------------------------------------------------- + + private ApplicationStoreEntry createApplicationStoreEntry(ApplicationID applicationId) { + return createApplicationStoreEntry(applicationId, "Test Application"); + } + + private ApplicationStoreEntry createApplicationStoreEntry( + ApplicationID applicationId, String name) { + return TestingApplicationStoreEntry.newBuilder() + .setApplicationId(applicationId) + .setName(name) + .build(); + } + + private void verifyApplicationStoreEntries( + ApplicationStoreEntry expected, ApplicationStoreEntry actual) { + assertThat(actual.getName()).isEqualTo(expected.getName()); + assertThat(actual.getApplicationId()).isEqualTo(expected.getApplicationId()); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingApplicationResultStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingApplicationResultStore.java new file mode 100644 index 0000000000000..cf6dae44f0f3b --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingApplicationResultStore.java @@ -0,0 +1,198 @@ +/* + * 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.ApplicationID; +import org.apache.flink.api.common.ApplicationState; +import org.apache.flink.runtime.highavailability.ApplicationResult; +import org.apache.flink.runtime.highavailability.ApplicationResultEntry; +import org.apache.flink.runtime.highavailability.ApplicationResultStore; +import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.function.SupplierWithException; + +import java.io.IOException; +import java.util.Collections; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; + +/** + * {@code TestingApplicationResultStore} is an {@link ApplicationResultStore} implementation that + * can be used in tests. + */ +public class TestingApplicationResultStore implements ApplicationResultStore { + + public static final ApplicationResult DUMMY_APPLICATION_RESULT = + createSuccessfulApplicationResult(new ApplicationID()); + + public static ApplicationResult createSuccessfulApplicationResult(ApplicationID applicationId) { + return createApplicationResult(applicationId, ApplicationState.FINISHED); + } + + public static ApplicationResult createApplicationResult( + ApplicationID applicationId, ApplicationState applicationState) { + return new ApplicationResult.Builder() + .applicationId(applicationId) + .applicationState(applicationState) + .applicationName("test-application") + .build(); + } + + private final Function> + createDirtyResultConsumer; + private final Function> markResultAsCleanConsumer; + + private final Function> + hasApplicationResultEntryFunction; + private final Function> + hasDirtyApplicationResultEntryFunction; + private final Function> + hasCleanApplicationResultEntryFunction; + private final SupplierWithException, ? extends IOException> + getDirtyResultsSupplier; + + private TestingApplicationResultStore( + Function> createDirtyResultConsumer, + Function> markResultAsCleanConsumer, + Function> hasApplicationResultEntryFunction, + Function> + hasDirtyApplicationResultEntryFunction, + Function> + hasCleanApplicationResultEntryFunction, + SupplierWithException, ? extends IOException> + getDirtyResultsSupplier) { + this.createDirtyResultConsumer = createDirtyResultConsumer; + this.markResultAsCleanConsumer = markResultAsCleanConsumer; + this.hasApplicationResultEntryFunction = hasApplicationResultEntryFunction; + this.hasDirtyApplicationResultEntryFunction = hasDirtyApplicationResultEntryFunction; + this.hasCleanApplicationResultEntryFunction = hasCleanApplicationResultEntryFunction; + this.getDirtyResultsSupplier = getDirtyResultsSupplier; + } + + @Override + public CompletableFuture createDirtyResultAsync( + ApplicationResultEntry applicationResultEntry) { + return createDirtyResultConsumer.apply(applicationResultEntry); + } + + @Override + public CompletableFuture markResultAsCleanAsync(ApplicationID applicationId) { + return markResultAsCleanConsumer.apply(applicationId); + } + + @Override + public CompletableFuture hasApplicationResultEntryAsync(ApplicationID applicationId) { + return hasApplicationResultEntryFunction.apply(applicationId); + } + + @Override + public CompletableFuture hasDirtyApplicationResultEntryAsync( + ApplicationID applicationId) { + return hasDirtyApplicationResultEntryFunction.apply(applicationId); + } + + @Override + public CompletableFuture hasCleanApplicationResultEntryAsync( + ApplicationID applicationId) { + return hasCleanApplicationResultEntryFunction.apply(applicationId); + } + + @Override + public Set getDirtyResults() throws IOException { + return getDirtyResultsSupplier.get(); + } + + public static TestingApplicationResultStore.Builder builder() { + return new Builder(); + } + + /** {@code Builder} for instantiating {@code TestingApplicationResultStore} instances. */ + public static class Builder { + + private Function> + createDirtyResultConsumer = + applicationResultEntry -> FutureUtils.completedVoidFuture(); + private Function> markResultAsCleanConsumer = + applicationID -> FutureUtils.completedVoidFuture(); + + private Function> + hasApplicationResultEntryFunction = + applicationID -> CompletableFuture.completedFuture(false); + private Function> + hasDirtyApplicationResultEntryFunction = + applicationID -> CompletableFuture.completedFuture(false); + private Function> + hasCleanApplicationResultEntryFunction = + applicationID -> CompletableFuture.completedFuture(false); + + private SupplierWithException, ? extends IOException> + getDirtyResultsSupplier = Collections::emptySet; + + public Builder withCreateDirtyResultConsumer( + Function> + createDirtyResultConsumer) { + this.createDirtyResultConsumer = createDirtyResultConsumer; + return this; + } + + public Builder withMarkResultAsCleanConsumer( + Function> markResultAsCleanConsumer) { + this.markResultAsCleanConsumer = markResultAsCleanConsumer; + return this; + } + + public Builder withHasApplicationResultEntryFunction( + Function> + hasApplicationResultEntryFunction) { + this.hasApplicationResultEntryFunction = hasApplicationResultEntryFunction; + return this; + } + + public Builder withHasDirtyApplicationResultEntryFunction( + Function> + hasDirtyApplicationResultEntryFunction) { + this.hasDirtyApplicationResultEntryFunction = hasDirtyApplicationResultEntryFunction; + return this; + } + + public Builder withHasCleanApplicationResultEntryFunction( + Function> + hasCleanApplicationResultEntryFunction) { + this.hasCleanApplicationResultEntryFunction = hasCleanApplicationResultEntryFunction; + return this; + } + + public Builder withGetDirtyResultsSupplier( + SupplierWithException, ? extends IOException> + getDirtyResultsSupplier) { + this.getDirtyResultsSupplier = getDirtyResultsSupplier; + return this; + } + + public TestingApplicationResultStore build() { + return new TestingApplicationResultStore( + createDirtyResultConsumer, + markResultAsCleanConsumer, + hasApplicationResultEntryFunction, + hasDirtyApplicationResultEntryFunction, + hasCleanApplicationResultEntryFunction, + getDirtyResultsSupplier); + } + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingApplicationStore.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingApplicationStore.java new file mode 100644 index 0000000000000..c937243cbfbe2 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingApplicationStore.java @@ -0,0 +1,263 @@ +/* + * 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.ApplicationID; +import org.apache.flink.runtime.jobmanager.ApplicationStore; +import org.apache.flink.runtime.jobmanager.ApplicationStoreEntry; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.util.function.BiFunctionWithException; +import org.apache.flink.util.function.FunctionWithException; +import org.apache.flink.util.function.ThrowingConsumer; +import org.apache.flink.util.function.ThrowingRunnable; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; +import java.util.function.BiFunction; + +/** In-Memory implementation of {@link ApplicationStore} for testing purposes. */ +public class TestingApplicationStore implements ApplicationStore { + + private final Map storedApplications = new HashMap<>(); + + private final ThrowingConsumer startConsumer; + + private final ThrowingRunnable stopRunnable; + + private final FunctionWithException< + Collection, Collection, ? extends Exception> + applicationIdsFunction; + + private final BiFunctionWithException< + ApplicationID, + Map, + ApplicationStoreEntry, + ? extends Exception> + recoverApplicationFunction; + + private final ThrowingConsumer + putApplicationConsumer; + + private final BiFunction> + globalCleanupFunction; + + private boolean started; + + private TestingApplicationStore( + ThrowingConsumer startConsumer, + ThrowingRunnable stopRunnable, + FunctionWithException< + Collection, + Collection, + ? extends Exception> + applicationIdsFunction, + BiFunctionWithException< + ApplicationID, + Map, + ApplicationStoreEntry, + ? extends Exception> + recoverApplicationFunction, + ThrowingConsumer putApplicationConsumer, + BiFunction> globalCleanupFunction, + Collection initialApplications) { + this.startConsumer = startConsumer; + this.stopRunnable = stopRunnable; + this.applicationIdsFunction = applicationIdsFunction; + this.recoverApplicationFunction = recoverApplicationFunction; + this.putApplicationConsumer = putApplicationConsumer; + this.globalCleanupFunction = globalCleanupFunction; + + for (ApplicationStoreEntry initialApplication : initialApplications) { + storedApplications.put(initialApplication.getApplicationId(), initialApplication); + } + } + + @Override + public synchronized void start() throws Exception { + startConsumer.accept(null); + started = true; + } + + @Override + public synchronized void stop() throws Exception { + stopRunnable.run(); + started = false; + } + + @Override + public synchronized Optional recoverApplication( + ApplicationID applicationId) throws Exception { + verifyIsStarted(); + return Optional.ofNullable( + recoverApplicationFunction.apply(applicationId, storedApplications)); + } + + @Override + public synchronized void putApplication(ApplicationStoreEntry applicationStoreEntry) + throws Exception { + verifyIsStarted(); + putApplicationConsumer.accept(applicationStoreEntry); + storedApplications.put(applicationStoreEntry.getApplicationId(), applicationStoreEntry); + } + + @Override + public synchronized CompletableFuture globalCleanupAsync( + ApplicationID applicationId, Executor executor) { + verifyIsStarted(); + return globalCleanupFunction + .apply(applicationId, executor) + .thenRun(() -> storedApplications.remove(applicationId)); + } + + @Override + public synchronized Collection getApplicationIds() throws Exception { + verifyIsStarted(); + return applicationIdsFunction.apply( + Collections.unmodifiableSet(new HashSet<>(storedApplications.keySet()))); + } + + public synchronized boolean contains(ApplicationID applicationId) { + return storedApplications.containsKey(applicationId); + } + + private void verifyIsStarted() { + Preconditions.checkState(started, "Not running. Forgot to call start()?"); + } + + public static Builder newBuilder() { + return new Builder(); + } + + /** {@code Builder} for creating {@code TestingApplicationStore} instances. */ + public static class Builder { + private ThrowingConsumer startConsumer = ignored -> {}; + + private ThrowingRunnable stopRunnable = () -> {}; + + private FunctionWithException< + Collection, Collection, ? extends Exception> + applicationIdsFunction = applicationIds -> applicationIds; + + private BiFunctionWithException< + ApplicationID, + Map, + ApplicationStoreEntry, + ? extends Exception> + recoverApplicationFunction = + (applicationId, applications) -> applications.get(applicationId); + + private ThrowingConsumer + putApplicationConsumer = ignored -> {}; + + private BiFunction> globalCleanupFunction = + (ignoredApplicationId, ignoredExecutor) -> FutureUtils.completedVoidFuture(); + + private Collection initialApplications = Collections.emptyList(); + + private boolean startApplicationStore = false; + + private Builder() {} + + public Builder setStartConsumer(ThrowingConsumer startConsumer) { + this.startConsumer = startConsumer; + return this; + } + + public Builder setStopRunnable(ThrowingRunnable stopRunnable) { + this.stopRunnable = stopRunnable; + return this; + } + + public Builder setApplicationIdsFunction( + FunctionWithException< + Collection, + Collection, + ? extends Exception> + applicationIdsFunction) { + this.applicationIdsFunction = applicationIdsFunction; + return this; + } + + public Builder setRecoverApplicationFunction( + BiFunctionWithException< + ApplicationID, + Map, + ApplicationStoreEntry, + ? extends Exception> + recoverApplicationFunction) { + this.recoverApplicationFunction = recoverApplicationFunction; + return this; + } + + public Builder setPutApplicationConsumer( + ThrowingConsumer + putApplicationConsumer) { + this.putApplicationConsumer = putApplicationConsumer; + return this; + } + + public Builder setGlobalCleanupFunction( + BiFunction> + globalCleanupFunction) { + this.globalCleanupFunction = globalCleanupFunction; + return this; + } + + public Builder setInitialApplications( + Collection initialApplications) { + this.initialApplications = initialApplications; + return this; + } + + public Builder withAutomaticStart() { + this.startApplicationStore = true; + return this; + } + + public TestingApplicationStore build() { + final TestingApplicationStore applicationStore = + new TestingApplicationStore( + startConsumer, + stopRunnable, + applicationIdsFunction, + recoverApplicationFunction, + putApplicationConsumer, + globalCleanupFunction, + initialApplications); + + if (startApplicationStore) { + try { + applicationStore.start(); + } catch (Exception e) { + ExceptionUtils.rethrow(e); + } + } + + return applicationStore; + } + } +}