diff --git a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8 b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8
index 05081f810d9ea..256be5208ffd8 100644
--- a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8
+++ b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8
@@ -21,7 +21,7 @@ org.apache.flink.cep.pattern.conditions.IterativeCondition.filter(java.lang.Obje
org.apache.flink.cep.pattern.conditions.SimpleCondition.filter(java.lang.Object, org.apache.flink.cep.pattern.conditions.IterativeCondition$Context): Argument leaf type org.apache.flink.cep.pattern.conditions.IterativeCondition$Context does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.client.program.StreamContextEnvironment.execute(org.apache.flink.streaming.api.graph.StreamGraph): Argument leaf type org.apache.flink.streaming.api.graph.StreamGraph does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.client.program.StreamContextEnvironment.executeAsync(org.apache.flink.streaming.api.graph.StreamGraph): Argument leaf type org.apache.flink.streaming.api.graph.StreamGraph does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
-org.apache.flink.client.program.StreamContextEnvironment.setAsContext(org.apache.flink.core.execution.PipelineExecutorServiceLoader, org.apache.flink.configuration.Configuration, java.lang.ClassLoader, boolean, boolean): Argument leaf type org.apache.flink.core.execution.PipelineExecutorServiceLoader does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
+org.apache.flink.client.program.StreamContextEnvironment.setAsContext(org.apache.flink.core.execution.PipelineExecutorServiceLoader, org.apache.flink.configuration.Configuration, java.lang.ClassLoader, boolean, boolean, org.apache.flink.api.common.ApplicationID): Argument leaf type org.apache.flink.core.execution.PipelineExecutorServiceLoader does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.client.program.StreamPlanEnvironment.executeAsync(org.apache.flink.streaming.api.graph.StreamGraph): Argument leaf type org.apache.flink.streaming.api.graph.StreamGraph does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.client.program.StreamPlanEnvironment.getPipeline(): Returned leaf type org.apache.flink.api.dag.Pipeline does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.configuration.ClusterOptions.getSchedulerType(org.apache.flink.configuration.Configuration): Returned leaf type org.apache.flink.configuration.JobManagerOptions$SchedulerType does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
diff --git a/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
index f8554ed3064a0..dd680883bc5fb 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
@@ -18,6 +18,7 @@
package org.apache.flink.client;
+import org.apache.flink.api.common.ApplicationID;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.client.cli.ClientOptions;
@@ -42,6 +43,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import javax.annotation.Nullable;
+
import java.net.URL;
import java.net.URLClassLoader;
import java.util.ArrayList;
@@ -80,6 +83,23 @@ public static void executeProgram(
boolean enforceSingleJobExecution,
boolean suppressSysout)
throws ProgramInvocationException {
+ executeProgram(
+ executorServiceLoader,
+ configuration,
+ program,
+ enforceSingleJobExecution,
+ suppressSysout,
+ null);
+ }
+
+ public static void executeProgram(
+ PipelineExecutorServiceLoader executorServiceLoader,
+ Configuration configuration,
+ PackagedProgram program,
+ boolean enforceSingleJobExecution,
+ boolean suppressSysout,
+ @Nullable ApplicationID applicationId)
+ throws ProgramInvocationException {
checkNotNull(executorServiceLoader);
final ClassLoader userCodeClassLoader = program.getUserCodeClassLoader();
final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader();
@@ -95,7 +115,8 @@ public static void executeProgram(
configuration,
userCodeClassLoader,
enforceSingleJobExecution,
- suppressSysout);
+ suppressSysout,
+ applicationId);
// For DataStream v2.
ExecutionContextEnvironment.setAsContext(
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrap.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrap.java
deleted file mode 100644
index 4290dcd474865..0000000000000
--- a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrap.java
+++ /dev/null
@@ -1,406 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.client.deployment.application;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.client.ClientUtils;
-import org.apache.flink.client.cli.ClientOptions;
-import org.apache.flink.client.deployment.application.executors.EmbeddedExecutor;
-import org.apache.flink.client.deployment.application.executors.EmbeddedExecutorServiceLoader;
-import org.apache.flink.client.program.PackagedProgram;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.DeploymentOptions;
-import org.apache.flink.configuration.HighAvailabilityOptions;
-import org.apache.flink.configuration.PipelineOptionsInternal;
-import org.apache.flink.core.execution.PipelineExecutorServiceLoader;
-import org.apache.flink.runtime.client.DuplicateJobSubmissionException;
-import org.apache.flink.runtime.clusterframework.ApplicationStatus;
-import org.apache.flink.runtime.dispatcher.DispatcherBootstrap;
-import org.apache.flink.runtime.dispatcher.DispatcherGateway;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
-import org.apache.flink.runtime.jobmaster.JobResult;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.messages.FlinkJobNotFoundException;
-import org.apache.flink.runtime.rpc.FatalErrorHandler;
-import org.apache.flink.util.ExceptionUtils;
-import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.concurrent.FutureUtils;
-import org.apache.flink.util.concurrent.ScheduledExecutor;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Optional;
-import java.util.Set;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CompletionException;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A {@link DispatcherBootstrap} used for running the user's {@code main()} in "Application Mode"
- * (see FLIP-85).
- *
- *
This dispatcher bootstrap submits the recovered {@link JobGraph job graphs} for re-execution
- * (in case of recovery from a failure), and then submits the remaining jobs of the application for
- * execution.
- *
- *
To achieve this, it works in conjunction with the {@link EmbeddedExecutor EmbeddedExecutor}
- * which decides if it should submit a job for execution (in case of a new job) or the job was
- * already recovered and is running.
- */
-@Internal
-public class ApplicationDispatcherBootstrap implements DispatcherBootstrap {
-
- @VisibleForTesting static final String FAILED_JOB_NAME = "(application driver)";
-
- private static final Logger LOG = LoggerFactory.getLogger(ApplicationDispatcherBootstrap.class);
-
- private static boolean isCanceledOrFailed(ApplicationStatus applicationStatus) {
- return applicationStatus == ApplicationStatus.CANCELED
- || applicationStatus == ApplicationStatus.FAILED;
- }
-
- private final PackagedProgram application;
-
- private final Collection recoveredJobIds;
-
- private final Configuration configuration;
-
- private final FatalErrorHandler errorHandler;
-
- private final CompletableFuture applicationCompletionFuture;
-
- private final CompletableFuture bootstrapCompletionFuture;
-
- private ScheduledFuture> applicationExecutionTask;
-
- public ApplicationDispatcherBootstrap(
- final PackagedProgram application,
- final Collection recoveredJobIds,
- final Configuration configuration,
- final DispatcherGateway dispatcherGateway,
- final ScheduledExecutor scheduledExecutor,
- final FatalErrorHandler errorHandler) {
- this.configuration = checkNotNull(configuration);
- this.recoveredJobIds = checkNotNull(recoveredJobIds);
- this.application = checkNotNull(application);
- this.errorHandler = checkNotNull(errorHandler);
-
- this.applicationCompletionFuture =
- fixJobIdAndRunApplicationAsync(dispatcherGateway, scheduledExecutor);
-
- this.bootstrapCompletionFuture = finishBootstrapTasks(dispatcherGateway);
- }
-
- @Override
- public void stop() {
- if (applicationExecutionTask != null) {
- applicationExecutionTask.cancel(true);
- }
-
- if (applicationCompletionFuture != null) {
- applicationCompletionFuture.cancel(true);
- }
- }
-
- @VisibleForTesting
- ScheduledFuture> getApplicationExecutionFuture() {
- return applicationExecutionTask;
- }
-
- @VisibleForTesting
- CompletableFuture getApplicationCompletionFuture() {
- return applicationCompletionFuture;
- }
-
- @VisibleForTesting
- CompletableFuture getBootstrapCompletionFuture() {
- return bootstrapCompletionFuture;
- }
-
- /**
- * Logs final application status and invokes error handler in case of unexpected failures.
- * Optionally shuts down the given dispatcherGateway when the application completes (either
- * successfully or in case of failure), depending on the corresponding config option.
- */
- private CompletableFuture finishBootstrapTasks(
- final DispatcherGateway dispatcherGateway) {
- final CompletableFuture shutdownFuture =
- applicationCompletionFuture
- .handle(
- (ignored, t) -> {
- if (t == null) {
- LOG.info("Application completed SUCCESSFULLY");
- return finish(
- dispatcherGateway, ApplicationStatus.SUCCEEDED);
- }
- final Optional maybeApplicationStatus =
- extractApplicationStatus(t);
- if (maybeApplicationStatus.isPresent()
- && isCanceledOrFailed(maybeApplicationStatus.get())) {
- final ApplicationStatus applicationStatus =
- maybeApplicationStatus.get();
- LOG.info("Application {}: ", applicationStatus, t);
- return finish(dispatcherGateway, applicationStatus);
- }
- if (t instanceof CancellationException) {
- LOG.warn(
- "Application has been cancelled because the {} is being stopped.",
- ApplicationDispatcherBootstrap.class
- .getSimpleName());
- return CompletableFuture.completedFuture(Acknowledge.get());
- }
- LOG.warn("Application failed unexpectedly: ", t);
- return FutureUtils.completedExceptionally(t);
- })
- .thenCompose(Function.identity());
- FutureUtils.handleUncaughtException(shutdownFuture, (t, e) -> errorHandler.onFatalError(e));
- return shutdownFuture;
- }
-
- private CompletableFuture finish(
- DispatcherGateway dispatcherGateway, ApplicationStatus applicationStatus) {
- boolean shouldShutDownOnFinish =
- configuration.get(DeploymentOptions.SHUTDOWN_ON_APPLICATION_FINISH);
- return shouldShutDownOnFinish
- ? dispatcherGateway.shutDownCluster(applicationStatus)
- : CompletableFuture.completedFuture(Acknowledge.get());
- }
-
- private Optional extractApplicationStatus(Throwable t) {
- final Optional maybeException =
- ExceptionUtils.findThrowable(t, UnsuccessfulExecutionException.class);
- return maybeException.map(
- exception -> ApplicationStatus.fromJobStatus(exception.getStatus().orElse(null)));
- }
-
- private CompletableFuture fixJobIdAndRunApplicationAsync(
- final DispatcherGateway dispatcherGateway, final ScheduledExecutor scheduledExecutor) {
- final Optional configuredJobId =
- configuration.getOptional(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID);
- final boolean submitFailedJobOnApplicationError =
- configuration.get(DeploymentOptions.SUBMIT_FAILED_JOB_ON_APPLICATION_ERROR);
- if (!HighAvailabilityMode.isHighAvailabilityModeActivated(configuration)
- && !configuredJobId.isPresent()) {
- return runApplicationAsync(
- dispatcherGateway, scheduledExecutor, false, submitFailedJobOnApplicationError);
- }
- if (!configuredJobId.isPresent()) {
- // In HA mode, we only support single-execute jobs at the moment. Here, we manually
- // generate the job id, if not configured, from the cluster id to keep it consistent
- // across failover.
- configuration.set(
- PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID,
- new JobID(
- Preconditions.checkNotNull(
- configuration.get(
- HighAvailabilityOptions.HA_CLUSTER_ID))
- .hashCode(),
- 0)
- .toHexString());
- }
- return runApplicationAsync(
- dispatcherGateway, scheduledExecutor, true, submitFailedJobOnApplicationError);
- }
-
- /**
- * Runs the user program entrypoint by scheduling a task on the given {@code scheduledExecutor}.
- * The returned {@link CompletableFuture} completes when all jobs of the user application
- * succeeded. if any of them fails, or if job submission fails.
- */
- private CompletableFuture runApplicationAsync(
- final DispatcherGateway dispatcherGateway,
- final ScheduledExecutor scheduledExecutor,
- final boolean enforceSingleJobExecution,
- final boolean submitFailedJobOnApplicationError) {
- final CompletableFuture> applicationExecutionFuture = new CompletableFuture<>();
- final Set tolerateMissingResult = Collections.synchronizedSet(new HashSet<>());
-
- // we need to hand in a future as return value because we need to get those JobIs out
- // from the scheduled task that executes the user program
- applicationExecutionTask =
- scheduledExecutor.schedule(
- () ->
- runApplicationEntryPoint(
- applicationExecutionFuture,
- tolerateMissingResult,
- dispatcherGateway,
- scheduledExecutor,
- enforceSingleJobExecution,
- submitFailedJobOnApplicationError),
- 0L,
- TimeUnit.MILLISECONDS);
-
- return applicationExecutionFuture.thenCompose(
- jobIds ->
- getApplicationResult(
- dispatcherGateway,
- jobIds,
- tolerateMissingResult,
- scheduledExecutor));
- }
-
- /**
- * Runs the user program entrypoint and completes the given {@code jobIdsFuture} with the {@link
- * JobID JobIDs} of the submitted jobs.
- *
- * This should be executed in a separate thread (or task).
- */
- private void runApplicationEntryPoint(
- final CompletableFuture> jobIdsFuture,
- final Set tolerateMissingResult,
- final DispatcherGateway dispatcherGateway,
- final ScheduledExecutor scheduledExecutor,
- final boolean enforceSingleJobExecution,
- final boolean submitFailedJobOnApplicationError) {
- if (submitFailedJobOnApplicationError && !enforceSingleJobExecution) {
- jobIdsFuture.completeExceptionally(
- new ApplicationExecutionException(
- String.format(
- "Submission of failed job in case of an application error ('%s') is not supported in non-HA setups.",
- DeploymentOptions.SUBMIT_FAILED_JOB_ON_APPLICATION_ERROR
- .key())));
- return;
- }
- final List applicationJobIds = new ArrayList<>(recoveredJobIds);
- try {
- final PipelineExecutorServiceLoader executorServiceLoader =
- new EmbeddedExecutorServiceLoader(
- applicationJobIds, dispatcherGateway, scheduledExecutor);
-
- ClientUtils.executeProgram(
- executorServiceLoader,
- configuration,
- application,
- enforceSingleJobExecution,
- true /* suppress sysout */);
-
- if (applicationJobIds.isEmpty()) {
- jobIdsFuture.completeExceptionally(
- new ApplicationExecutionException(
- "The application contains no execute() calls."));
- } else {
- jobIdsFuture.complete(applicationJobIds);
- }
- } catch (Throwable t) {
- // If we're running in a single job execution mode, it's safe to consider re-submission
- // of an already finished a success.
- final Optional maybeDuplicate =
- ExceptionUtils.findThrowable(t, DuplicateJobSubmissionException.class);
- if (enforceSingleJobExecution
- && maybeDuplicate.isPresent()
- && maybeDuplicate.get().isGloballyTerminated()) {
- final JobID jobId = maybeDuplicate.get().getJobID();
- tolerateMissingResult.add(jobId);
- jobIdsFuture.complete(Collections.singletonList(jobId));
- } else if (submitFailedJobOnApplicationError && applicationJobIds.isEmpty()) {
- final JobID failedJobId =
- JobID.fromHexString(
- configuration.get(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID));
- dispatcherGateway
- .submitFailedJob(failedJobId, FAILED_JOB_NAME, t)
- .thenAccept(
- ignored ->
- jobIdsFuture.complete(
- Collections.singletonList(failedJobId)));
- } else {
- jobIdsFuture.completeExceptionally(
- new ApplicationExecutionException("Could not execute application.", t));
- }
- }
- }
-
- private CompletableFuture getApplicationResult(
- final DispatcherGateway dispatcherGateway,
- final Collection applicationJobIds,
- final Set tolerateMissingResult,
- final ScheduledExecutor executor) {
- final List> jobResultFutures =
- applicationJobIds.stream()
- .map(
- jobId ->
- unwrapJobResultException(
- getJobResult(
- dispatcherGateway,
- jobId,
- executor,
- tolerateMissingResult.contains(jobId))))
- .collect(Collectors.toList());
- return FutureUtils.waitForAll(jobResultFutures);
- }
-
- private CompletableFuture getJobResult(
- final DispatcherGateway dispatcherGateway,
- final JobID jobId,
- final ScheduledExecutor scheduledExecutor,
- final boolean tolerateMissingResult) {
- final Duration timeout = configuration.get(ClientOptions.CLIENT_TIMEOUT);
- final Duration retryPeriod = configuration.get(ClientOptions.CLIENT_RETRY_PERIOD);
- final CompletableFuture jobResultFuture =
- JobStatusPollingUtils.getJobResult(
- dispatcherGateway, jobId, scheduledExecutor, timeout, retryPeriod);
- if (tolerateMissingResult) {
- // Return "unknown" job result if dispatcher no longer knows the actual result.
- return FutureUtils.handleException(
- jobResultFuture,
- FlinkJobNotFoundException.class,
- exception ->
- new JobResult.Builder()
- .jobId(jobId)
- .jobStatus(null)
- .netRuntime(Long.MAX_VALUE)
- .build());
- }
- return jobResultFuture;
- }
-
- /**
- * If the given {@link JobResult} indicates success, this passes through the {@link JobResult}.
- * Otherwise, this returns a future that is finished exceptionally (potentially with an
- * exception from the {@link JobResult}).
- */
- private CompletableFuture unwrapJobResultException(
- final CompletableFuture jobResult) {
- return jobResult.thenApply(
- result -> {
- if (result.isSuccess()) {
- return result;
- }
-
- throw new CompletionException(
- UnsuccessfulExecutionException.fromJobResult(
- result, application.getUserCodeClassLoader()));
- });
- }
-}
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 b7e188ef7d49b..6c443c4b64955 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
@@ -19,9 +19,13 @@
package org.apache.flink.client.deployment.application;
import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.ApplicationID;
import org.apache.flink.api.common.JobID;
import org.apache.flink.client.program.PackagedProgram;
+import org.apache.flink.configuration.ApplicationOptionsInternal;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.DeploymentOptions;
+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;
@@ -51,8 +55,7 @@
*
* It instantiates a {@link
* org.apache.flink.runtime.dispatcher.runner.AbstractDispatcherLeaderProcess.DispatcherGatewayService
- * DispatcherGatewayService} with an {@link ApplicationDispatcherBootstrap} containing the user's
- * program.
+ * DispatcherGatewayService} with an {@link ApplicationBootstrap} containing the user's program.
*/
@Internal
public class ApplicationDispatcherGatewayServiceFactory
@@ -62,7 +65,7 @@ public class ApplicationDispatcherGatewayServiceFactory
private final DispatcherFactory dispatcherFactory;
- private final PackagedProgram application;
+ private final PackagedProgram program;
private final RpcService rpcService;
@@ -71,12 +74,12 @@ public class ApplicationDispatcherGatewayServiceFactory
public ApplicationDispatcherGatewayServiceFactory(
Configuration configuration,
DispatcherFactory dispatcherFactory,
- PackagedProgram application,
+ PackagedProgram program,
RpcService rpcService,
PartialDispatcherServices partialDispatcherServices) {
this.configuration = configuration;
this.dispatcherFactory = dispatcherFactory;
- this.application = checkNotNull(application);
+ this.program = checkNotNull(program);
this.rpcService = rpcService;
this.partialDispatcherServices = partialDispatcherServices;
}
@@ -91,6 +94,26 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create(
final List recoveredJobIds = getRecoveredJobIds(recoveredJobs);
+ final boolean allowExecuteMultipleJobs =
+ ApplicationJobUtils.allowExecuteMultipleJobs(configuration);
+ ApplicationJobUtils.maybeFixIds(configuration);
+ final ApplicationID applicationId =
+ configuration
+ .getOptional(ApplicationOptionsInternal.FIXED_APPLICATION_ID)
+ .map(ApplicationID::fromHexString)
+ .orElseGet(ApplicationID::new);
+
+ PackagedProgramApplication bootstrapApplication =
+ new PackagedProgramApplication(
+ applicationId,
+ program,
+ recoveredJobIds,
+ configuration,
+ true,
+ !allowExecuteMultipleJobs,
+ configuration.get(DeploymentOptions.SUBMIT_FAILED_JOB_ON_APPLICATION_ERROR),
+ configuration.get(DeploymentOptions.SHUTDOWN_ON_APPLICATION_FINISH));
+
final Dispatcher dispatcher;
try {
dispatcher =
@@ -100,13 +123,7 @@ public AbstractDispatcherLeaderProcess.DispatcherGatewayService create(
recoveredJobs,
recoveredDirtyJobResults,
(dispatcherGateway, scheduledExecutor, errorHandler) ->
- new ApplicationDispatcherBootstrap(
- application,
- recoveredJobIds,
- configuration,
- dispatcherGateway,
- scheduledExecutor,
- errorHandler),
+ new ApplicationBootstrap(bootstrapApplication),
PartialDispatcherServicesWithJobPersistenceComponents.from(
partialDispatcherServices,
executionPlanWriter,
diff --git a/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationJobUtils.java b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationJobUtils.java
new file mode 100644
index 0000000000000..c80055ebe79b7
--- /dev/null
+++ b/flink-clients/src/main/java/org/apache/flink/client/deployment/application/ApplicationJobUtils.java
@@ -0,0 +1,91 @@
+/*
+ * 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.JobID;
+import org.apache.flink.configuration.ApplicationOptionsInternal;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.PipelineOptionsInternal;
+import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
+import org.apache.flink.util.Preconditions;
+
+import java.util.Optional;
+
+/** Utility class to handle application/job related configuration options in application mode. */
+public class ApplicationJobUtils {
+
+ public static void maybeFixIds(Configuration configuration) {
+ if (HighAvailabilityMode.isHighAvailabilityModeActivated(configuration)) {
+ final Optional configuredApplicationId =
+ configuration.getOptional(ApplicationOptionsInternal.FIXED_APPLICATION_ID);
+ if (configuredApplicationId.isEmpty()) {
+ // In HA mode, a fixed application id is required to ensure consistency across
+ // failovers. The application id is derived from the cluster id.
+ configuration.set(
+ ApplicationOptionsInternal.FIXED_APPLICATION_ID,
+ new ApplicationID(
+ Preconditions.checkNotNull(
+ configuration.get(
+ HighAvailabilityOptions
+ .HA_CLUSTER_ID))
+ .hashCode(),
+ 0)
+ .toHexString());
+ }
+ final Optional configuredJobId =
+ configuration.getOptional(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID);
+ if (configuredJobId.isEmpty()) {
+ // In HA mode, a fixed job id is required to ensure consistency across failovers.
+ // The job id is derived as follows:
+ // 1. If application id is configured, use the application id as the job id.
+ // 2. Otherwise, generate the job id based on the HA cluster id.
+ // Note that the second case is kept for backward compatibility and may be removed.
+ if (configuredApplicationId.isPresent()) {
+ ApplicationID applicationId =
+ ApplicationID.fromHexString(
+ configuration.get(
+ ApplicationOptionsInternal.FIXED_APPLICATION_ID));
+ configuration.set(
+ PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID,
+ applicationId.toHexString());
+ } else {
+ configuration.set(
+ PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID,
+ new JobID(
+ Preconditions.checkNotNull(
+ configuration.get(
+ HighAvailabilityOptions
+ .HA_CLUSTER_ID))
+ .hashCode(),
+ 0)
+ .toHexString());
+ }
+ }
+ }
+ }
+
+ public static boolean allowExecuteMultipleJobs(Configuration config) {
+ final Optional configuredJobId =
+ config.getOptional(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID);
+ return !HighAvailabilityMode.isHighAvailabilityModeActivated(config)
+ && !configuredJobId.isPresent();
+ }
+}
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 5cc2f2cc04749..de5de97b7526d 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
@@ -533,7 +533,8 @@ private void runApplicationEntryPoint(
configuration,
program,
enforceSingleJobExecution,
- true /* suppress sysout */);
+ true /* suppress sysout */,
+ getApplicationId());
if (applicationJobIds.isEmpty()) {
jobIdsFuture.completeExceptionally(
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java
index 7dde266cf8528..2799ab63cb5db 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java
@@ -19,6 +19,7 @@
import org.apache.flink.annotation.Internal;
import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.ApplicationID;
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.client.ClientUtils;
import org.apache.flink.client.cli.ClientOptions;
@@ -42,6 +43,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import javax.annotation.Nullable;
+
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@@ -72,6 +75,8 @@ public class StreamContextEnvironment extends StreamExecutionEnvironment {
private final boolean programConfigEnabled;
private final Collection programConfigWildcards;
+ @Nullable private final ApplicationID applicationId;
+
public StreamContextEnvironment(
final PipelineExecutorServiceLoader executorServiceLoader,
final Configuration configuration,
@@ -89,7 +94,6 @@ public StreamContextEnvironment(
Collections.emptyList());
}
- @Internal
public StreamContextEnvironment(
final PipelineExecutorServiceLoader executorServiceLoader,
final Configuration clusterConfiguration,
@@ -99,6 +103,29 @@ public StreamContextEnvironment(
final boolean suppressSysout,
final boolean programConfigEnabled,
final Collection programConfigWildcards) {
+ this(
+ executorServiceLoader,
+ clusterConfiguration,
+ configuration,
+ userCodeClassLoader,
+ enforceSingleJobExecution,
+ suppressSysout,
+ programConfigEnabled,
+ programConfigWildcards,
+ null);
+ }
+
+ @Internal
+ public StreamContextEnvironment(
+ final PipelineExecutorServiceLoader executorServiceLoader,
+ final Configuration clusterConfiguration,
+ final Configuration configuration,
+ final ClassLoader userCodeClassLoader,
+ final boolean enforceSingleJobExecution,
+ final boolean suppressSysout,
+ final boolean programConfigEnabled,
+ final Collection programConfigWildcards,
+ @Nullable final ApplicationID applicationId) {
super(executorServiceLoader, configuration, userCodeClassLoader);
this.suppressSysout = suppressSysout;
this.enforceSingleJobExecution = enforceSingleJobExecution;
@@ -106,6 +133,7 @@ public StreamContextEnvironment(
this.jobCounter = 0;
this.programConfigEnabled = programConfigEnabled;
this.programConfigWildcards = programConfigWildcards;
+ this.applicationId = applicationId;
}
@Override
@@ -185,6 +213,9 @@ private JobExecutionResult getJobExecutionResult(final JobClient jobClient) thro
public JobClient executeAsync(StreamGraph streamGraph) throws Exception {
checkNotAllowedConfigurations();
validateAllowedExecution();
+ if (applicationId != null) {
+ streamGraph.setApplicationId(applicationId);
+ }
final JobClient jobClient = super.executeAsync(streamGraph);
if (!suppressSysout) {
@@ -209,7 +240,8 @@ public static void setAsContext(
final Configuration clusterConfiguration,
final ClassLoader userCodeClassLoader,
final boolean enforceSingleJobExecution,
- final boolean suppressSysout) {
+ final boolean suppressSysout,
+ @Nullable final ApplicationID applicationId) {
final StreamExecutionEnvironmentFactory factory =
envInitConfig -> {
final boolean programConfigEnabled =
@@ -227,7 +259,8 @@ public static void setAsContext(
enforceSingleJobExecution,
suppressSysout,
programConfigEnabled,
- programConfigWildcards);
+ programConfigWildcards,
+ applicationId);
};
initializeContextEnvironment(factory);
}
diff --git a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapTest.java b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapTest.java
deleted file mode 100644
index 47c5e13fe7bbd..0000000000000
--- a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapTest.java
+++ /dev/null
@@ -1,1023 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.client.deployment.application;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.JobStatus;
-import org.apache.flink.client.deployment.application.executors.EmbeddedExecutor;
-import org.apache.flink.client.program.PackagedProgram;
-import org.apache.flink.client.program.ProgramInvocationException;
-import org.apache.flink.client.testjar.FailingJob;
-import org.apache.flink.client.testjar.MultiExecuteJob;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.DeploymentOptions;
-import org.apache.flink.configuration.HighAvailabilityOptions;
-import org.apache.flink.configuration.PipelineOptionsInternal;
-import org.apache.flink.runtime.client.DuplicateJobSubmissionException;
-import org.apache.flink.runtime.client.JobCancellationException;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.runtime.clusterframework.ApplicationStatus;
-import org.apache.flink.runtime.dispatcher.DispatcherGateway;
-import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
-import org.apache.flink.runtime.jobmaster.JobResult;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.messages.FlinkJobNotFoundException;
-import org.apache.flink.runtime.rpc.FatalErrorHandler;
-import org.apache.flink.runtime.webmonitor.TestingDispatcherGateway;
-import org.apache.flink.util.ExceptionUtils;
-import org.apache.flink.util.ExecutorUtils;
-import org.apache.flink.util.FlinkException;
-import org.apache.flink.util.FlinkRuntimeException;
-import org.apache.flink.util.SerializedThrowable;
-import org.apache.flink.util.concurrent.FutureUtils;
-import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor;
-import org.apache.flink.util.concurrent.ScheduledExecutor;
-import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter;
-
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.params.ParameterizedTest;
-import org.junit.jupiter.params.provider.EnumSource;
-
-import javax.annotation.Nullable;
-
-import java.util.Collections;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentLinkedDeque;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.BiConsumer;
-import java.util.function.Supplier;
-
-import static org.apache.flink.core.testutils.FlinkAssertions.assertThatFuture;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.fail;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-
-/** Tests for the {@link ApplicationDispatcherBootstrap}. */
-class ApplicationDispatcherBootstrapTest {
-
- private static final int TIMEOUT_SECONDS = 10;
-
- private final ScheduledExecutorService executor = Executors.newScheduledThreadPool(4);
- private final ScheduledExecutor scheduledExecutor =
- new ScheduledExecutorServiceAdapter(executor);
-
- @AfterEach
- void cleanup() {
- ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
- }
-
- @Test
- void testExceptionThrownWhenApplicationContainsNoJobs() throws Throwable {
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- TestingDispatcherGateway.newBuilder()
- .setSubmitFunction(
- jobGraph -> CompletableFuture.completedFuture(Acknowledge.get()));
-
- final CompletableFuture applicationFuture = runApplication(dispatcherBuilder, 0);
-
- assertException(applicationFuture, ApplicationExecutionException.class);
- }
-
- @Test
- void testOnlyOneJobIsAllowedWithHa() throws Throwable {
- final Configuration configurationUnderTest = getConfiguration();
- configurationUnderTest.set(
- HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name());
-
- final CompletableFuture applicationFuture = runApplication(configurationUnderTest, 2);
-
- assertException(applicationFuture, FlinkRuntimeException.class);
- }
-
- @Test
- void testOnlyOneJobAllowedWithStaticJobId() throws Throwable {
- final JobID testJobID = new JobID(0, 2);
-
- final Configuration configurationUnderTest = getConfiguration();
- configurationUnderTest.set(
- PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID, testJobID.toHexString());
-
- final CompletableFuture applicationFuture = runApplication(configurationUnderTest, 2);
-
- assertException(applicationFuture, FlinkRuntimeException.class);
- }
-
- @Test
- void testOnlyOneJobAllowedWithStaticJobIdAndHa() throws Throwable {
- final JobID testJobID = new JobID(0, 2);
-
- final Configuration configurationUnderTest = getConfiguration();
- configurationUnderTest.set(
- PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID, testJobID.toHexString());
- configurationUnderTest.set(
- HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name());
-
- final CompletableFuture applicationFuture = runApplication(configurationUnderTest, 2);
-
- assertException(applicationFuture, FlinkRuntimeException.class);
- }
-
- @Test
- void testJobIdDefaultsToClusterIdWithHa() throws Throwable {
- final Configuration configurationUnderTest = getConfiguration();
- final String clusterId = "cluster";
- configurationUnderTest.set(
- HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name());
- configurationUnderTest.set(HighAvailabilityOptions.HA_CLUSTER_ID, clusterId);
-
- final CompletableFuture submittedJobId = new CompletableFuture<>();
-
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- finishedJobGatewayBuilder()
- .setSubmitFunction(
- jobGraph -> {
- submittedJobId.complete(jobGraph.getJobID());
- return CompletableFuture.completedFuture(Acknowledge.get());
- });
-
- final CompletableFuture applicationFuture =
- runApplication(dispatcherBuilder, configurationUnderTest, 1);
-
- applicationFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
-
- assertThat(submittedJobId.get(TIMEOUT_SECONDS, TimeUnit.SECONDS))
- .isEqualTo(new JobID(clusterId.hashCode(), 0L));
- }
-
- @Test
- void testStaticJobId() throws Throwable {
- final JobID testJobID = new JobID(0, 2);
-
- final Configuration configurationUnderTest = getConfiguration();
- configurationUnderTest.set(
- PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID, testJobID.toHexString());
-
- final CompletableFuture submittedJobId = new CompletableFuture<>();
-
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- finishedJobGatewayBuilder()
- .setSubmitFunction(
- jobGraph -> {
- submittedJobId.complete(jobGraph.getJobID());
- return CompletableFuture.completedFuture(Acknowledge.get());
- });
-
- final CompletableFuture applicationFuture =
- runApplication(dispatcherBuilder, configurationUnderTest, 1);
-
- applicationFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
-
- assertThat(submittedJobId.get(TIMEOUT_SECONDS, TimeUnit.SECONDS))
- .isEqualTo(new JobID(0L, 2L));
- }
-
- @Test
- void testStaticJobIdWithHa() throws Throwable {
- final JobID testJobID = new JobID(0, 2);
-
- final Configuration configurationUnderTest = getConfiguration();
- configurationUnderTest.set(
- PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID, testJobID.toHexString());
- configurationUnderTest.set(
- HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name());
-
- final CompletableFuture submittedJobId = new CompletableFuture<>();
-
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- finishedJobGatewayBuilder()
- .setSubmitFunction(
- jobGraph -> {
- submittedJobId.complete(jobGraph.getJobID());
- return CompletableFuture.completedFuture(Acknowledge.get());
- });
-
- final CompletableFuture applicationFuture =
- runApplication(dispatcherBuilder, configurationUnderTest, 1);
-
- applicationFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
-
- assertThat(submittedJobId.get(TIMEOUT_SECONDS, TimeUnit.SECONDS))
- .isEqualTo(new JobID(0L, 2L));
- }
-
- @Test
- void testApplicationFailsAsSoonAsOneJobFails() throws Throwable {
- final ConcurrentLinkedDeque submittedJobIds = new ConcurrentLinkedDeque<>();
-
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- TestingDispatcherGateway.newBuilder()
- .setSubmitFunction(
- jobGraph -> {
- submittedJobIds.add(jobGraph.getJobID());
- return CompletableFuture.completedFuture(Acknowledge.get());
- })
- .setRequestJobStatusFunction(
- jobId -> {
- // we only fail one of the jobs, the first one, the others will
- // "keep" running
- // indefinitely
- if (jobId.equals(submittedJobIds.peek())) {
- return CompletableFuture.completedFuture(JobStatus.FAILED);
- }
- // never finish the other jobs
- return CompletableFuture.completedFuture(JobStatus.RUNNING);
- })
- .setRequestJobResultFunction(
- jobId -> {
- // we only fail one of the jobs, the first one, the other will
- // "keep" running
- // indefinitely. If we didn't have this the test would hang
- // forever.
- if (jobId.equals(submittedJobIds.peek())) {
- return CompletableFuture.completedFuture(
- createFailedJobResult(jobId));
- }
- // never finish the other jobs
- return new CompletableFuture<>();
- });
-
- final CompletableFuture applicationFuture = runApplication(dispatcherBuilder, 2);
- final UnsuccessfulExecutionException exception =
- assertException(applicationFuture, UnsuccessfulExecutionException.class);
- assertThat(exception.getStatus().orElse(null)).isEqualTo(JobStatus.FAILED);
- }
-
- @Test
- void testApplicationSucceedsWhenAllJobsSucceed() throws Exception {
- final TestingDispatcherGateway.Builder dispatcherBuilder = finishedJobGatewayBuilder();
-
- final CompletableFuture applicationFuture = runApplication(dispatcherBuilder, 3);
-
- // this would block indefinitely if the applications don't finish
- applicationFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
- }
-
- @Test
- void testDispatcherIsCancelledWhenOneJobIsCancelled() throws Exception {
- final CompletableFuture clusterShutdownStatus =
- new CompletableFuture<>();
-
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- canceledJobGatewayBuilder()
- .setClusterShutdownFunction(
- status -> {
- clusterShutdownStatus.complete(status);
- return CompletableFuture.completedFuture(Acknowledge.get());
- });
-
- ApplicationDispatcherBootstrap bootstrap =
- createApplicationDispatcherBootstrap(
- 3, dispatcherBuilder.build(), scheduledExecutor);
-
- final CompletableFuture completionFuture =
- bootstrap.getBootstrapCompletionFuture();
-
- // wait until the bootstrap "thinks" it's done, also makes sure that we don't
- // fail the future exceptionally with a JobCancelledException
- completionFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
-
- assertThat(clusterShutdownStatus.get(TIMEOUT_SECONDS, TimeUnit.SECONDS))
- .isEqualTo(ApplicationStatus.CANCELED);
- }
-
- @Test
- void testApplicationTaskFinishesWhenApplicationFinishes() throws Exception {
- final TestingDispatcherGateway.Builder dispatcherBuilder = finishedJobGatewayBuilder();
-
- ApplicationDispatcherBootstrap bootstrap =
- createApplicationDispatcherBootstrap(
- 3, dispatcherBuilder.build(), scheduledExecutor);
-
- final CompletableFuture completionFuture =
- bootstrap.getBootstrapCompletionFuture();
-
- ScheduledFuture> applicationExecutionFuture = bootstrap.getApplicationExecutionFuture();
-
- // wait until the bootstrap "thinks" it's done
- completionFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
-
- // make sure the task finishes
- applicationExecutionFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
- }
-
- @Test
- void testApplicationIsStoppedWhenStoppingBootstrap() throws Exception {
- final AtomicBoolean shutdownCalled = new AtomicBoolean(false);
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- runningJobGatewayBuilder()
- .setClusterShutdownFunction(
- status -> {
- shutdownCalled.set(true);
- return CompletableFuture.completedFuture(Acknowledge.get());
- });
-
- final ManuallyTriggeredScheduledExecutor manuallyTriggeredExecutor =
- new ManuallyTriggeredScheduledExecutor();
- // we're "listening" on this to be completed to verify that the error handler is called.
- // In production, this will shut down the cluster with an exception.
- final CompletableFuture errorHandlerFuture = new CompletableFuture<>();
- final ApplicationDispatcherBootstrap bootstrap =
- createApplicationDispatcherBootstrap(
- 3,
- dispatcherBuilder.build(),
- manuallyTriggeredExecutor,
- errorHandlerFuture::completeExceptionally);
-
- final CompletableFuture completionFuture =
- bootstrap.getBootstrapCompletionFuture();
-
- ScheduledFuture> applicationExecutionFuture = bootstrap.getApplicationExecutionFuture();
-
- bootstrap.stop();
-
- // Triggers the scheduled ApplicationDispatcherBootstrap process after calling stop. This
- // ensures that the bootstrap task isn't completed before the stop method is called which
- // would prevent the stop call from cancelling the task's future.
- manuallyTriggeredExecutor.triggerNonPeriodicScheduledTask();
-
- // we didn't call the error handler
- assertThat(errorHandlerFuture.isDone()).isFalse();
-
- // completion future gets completed normally
- completionFuture.get();
-
- // verify that we didn't shut down the cluster
- assertThat(shutdownCalled.get()).isFalse();
-
- // verify that the application task is being cancelled
- assertThat(applicationExecutionFuture.isCancelled()).isTrue();
- assertThat(applicationExecutionFuture.isDone()).isTrue();
- }
-
- @Test
- void testErrorHandlerIsCalledWhenSubmissionThrowsAnException() throws Exception {
- final AtomicBoolean shutdownCalled = new AtomicBoolean(false);
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- runningJobGatewayBuilder()
- .setSubmitFunction(
- jobGraph -> {
- throw new FlinkRuntimeException("Nope!");
- })
- .setClusterShutdownFunction(
- status -> {
- shutdownCalled.set(true);
- return CompletableFuture.completedFuture(Acknowledge.get());
- });
-
- // we're "listening" on this to be completed to verify that the error handler is called.
- // In production, this will shut down the cluster with an exception.
- final CompletableFuture errorHandlerFuture = new CompletableFuture<>();
- final ApplicationDispatcherBootstrap bootstrap =
- createApplicationDispatcherBootstrap(
- 2,
- dispatcherBuilder.build(),
- scheduledExecutor,
- errorHandlerFuture::completeExceptionally);
-
- final CompletableFuture completionFuture =
- bootstrap.getBootstrapCompletionFuture();
-
- // we call the error handler
- assertException(errorHandlerFuture, FlinkRuntimeException.class);
-
- // we return a future that is completed exceptionally
- assertException(completionFuture, FlinkRuntimeException.class);
-
- // and cluster shutdown didn't get called
- assertThat(shutdownCalled.get()).isFalse();
- }
-
- @Test
- void testErrorHandlerIsCalledWhenShutdownCompletesExceptionally() throws Exception {
- testErrorHandlerIsCalled(
- () ->
- FutureUtils.completedExceptionally(
- new FlinkRuntimeException("Test exception.")));
- }
-
- @Test
- void testErrorHandlerIsCalledWhenShutdownThrowsAnException() throws Exception {
- testErrorHandlerIsCalled(
- () -> {
- throw new FlinkRuntimeException("Test exception.");
- });
- }
-
- private void testErrorHandlerIsCalled(Supplier> shutdownFunction)
- throws Exception {
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- TestingDispatcherGateway.newBuilder()
- .setSubmitFunction(
- jobGraph -> CompletableFuture.completedFuture(Acknowledge.get()))
- .setRequestJobStatusFunction(
- jobId -> CompletableFuture.completedFuture(JobStatus.FINISHED))
- .setRequestJobResultFunction(
- jobId ->
- CompletableFuture.completedFuture(
- createJobResult(jobId, JobStatus.FINISHED)))
- .setClusterShutdownFunction(status -> shutdownFunction.get());
-
- // we're "listening" on this to be completed to verify that the error handler is called.
- // In production, this will shut down the cluster with an exception.
- final CompletableFuture errorHandlerFuture = new CompletableFuture<>();
- final TestingDispatcherGateway dispatcherGateway = dispatcherBuilder.build();
- final ApplicationDispatcherBootstrap bootstrap =
- createApplicationDispatcherBootstrap(
- 3,
- dispatcherGateway,
- scheduledExecutor,
- errorHandlerFuture::completeExceptionally);
-
- final CompletableFuture completionFuture =
- bootstrap.getBootstrapCompletionFuture();
-
- // we call the error handler
- assertException(errorHandlerFuture, FlinkRuntimeException.class);
-
- // we return a future that is completed exceptionally
- assertException(completionFuture, FlinkRuntimeException.class);
- }
-
- @Test
- void testClusterIsShutdownInAttachedModeWhenJobCancelled() throws Exception {
- final CompletableFuture clusterShutdown = new CompletableFuture<>();
-
- final TestingDispatcherGateway dispatcherGateway =
- canceledJobGatewayBuilder()
- .setClusterShutdownFunction(
- status -> {
- clusterShutdown.complete(status);
- return CompletableFuture.completedFuture(Acknowledge.get());
- })
- .build();
-
- final PackagedProgram program = getProgram(2);
-
- final Configuration configuration = getConfiguration();
- configuration.set(DeploymentOptions.ATTACHED, true);
-
- final ApplicationDispatcherBootstrap bootstrap =
- new ApplicationDispatcherBootstrap(
- program,
- Collections.emptyList(),
- configuration,
- dispatcherGateway,
- scheduledExecutor,
- e -> {});
-
- final CompletableFuture applicationFuture =
- bootstrap.getApplicationCompletionFuture();
- assertException(applicationFuture, UnsuccessfulExecutionException.class);
-
- assertThat(clusterShutdown.get()).isEqualTo(ApplicationStatus.CANCELED);
- }
-
- @Test
- void testClusterShutdownWhenApplicationSucceeds() throws Exception {
- // we're "listening" on this to be completed to verify that the cluster
- // is being shut down from the ApplicationDispatcherBootstrap
- final CompletableFuture externalShutdownFuture =
- new CompletableFuture<>();
-
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- finishedJobGatewayBuilder()
- .setClusterShutdownFunction(
- status -> {
- externalShutdownFuture.complete(status);
- return CompletableFuture.completedFuture(Acknowledge.get());
- });
-
- ApplicationDispatcherBootstrap bootstrap =
- createApplicationDispatcherBootstrap(
- 3, dispatcherBuilder.build(), scheduledExecutor);
-
- final CompletableFuture completionFuture =
- bootstrap.getBootstrapCompletionFuture();
-
- // wait until the bootstrap "thinks" it's done
- completionFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
-
- // verify that the dispatcher is actually being shut down
- assertThat(externalShutdownFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS))
- .isEqualTo(ApplicationStatus.SUCCEEDED);
- }
-
- @Test
- void testClusterShutdownWhenApplicationFails() throws Exception {
- // we're "listening" on this to be completed to verify that the cluster
- // is being shut down from the ApplicationDispatcherBootstrap
- final CompletableFuture externalShutdownFuture =
- new CompletableFuture<>();
-
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- failedJobGatewayBuilder()
- .setClusterShutdownFunction(
- status -> {
- externalShutdownFuture.complete(status);
- return CompletableFuture.completedFuture(Acknowledge.get());
- });
-
- ApplicationDispatcherBootstrap bootstrap =
- createApplicationDispatcherBootstrap(
- 3, dispatcherBuilder.build(), scheduledExecutor);
-
- final CompletableFuture completionFuture =
- bootstrap.getBootstrapCompletionFuture();
-
- // wait until the bootstrap "thinks" it's done
- completionFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
-
- // verify that the dispatcher is actually being shut down
- assertThat(externalShutdownFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS))
- .isEqualTo(ApplicationStatus.FAILED);
- }
-
- @Test
- void testClusterShutdownWhenApplicationGetsCancelled() throws Exception {
- // we're "listening" on this to be completed to verify that the cluster
- // is being shut down from the ApplicationDispatcherBootstrap
- final CompletableFuture externalShutdownFuture =
- new CompletableFuture<>();
-
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- canceledJobGatewayBuilder()
- .setClusterShutdownFunction(
- status -> {
- externalShutdownFuture.complete(status);
- return CompletableFuture.completedFuture(Acknowledge.get());
- });
-
- ApplicationDispatcherBootstrap bootstrap =
- createApplicationDispatcherBootstrap(
- 3, dispatcherBuilder.build(), scheduledExecutor);
-
- final CompletableFuture completionFuture =
- bootstrap.getBootstrapCompletionFuture();
-
- // wait until the bootstrap "thinks" it's done
- completionFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
-
- // verify that the dispatcher is actually being shut down
- assertThat(externalShutdownFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS))
- .isEqualTo(ApplicationStatus.CANCELED);
- }
-
- @Test
- void testErrorHandlerIsCalledWhenApplicationStatusIsUnknown() throws Exception {
- // we're "listening" on this to be completed to verify that the cluster
- // is being shut down from the ApplicationDispatcherBootstrap
- final AtomicBoolean shutdownCalled = new AtomicBoolean(false);
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- canceledJobGatewayBuilder()
- .setRequestJobResultFunction(
- jobID ->
- CompletableFuture.completedFuture(
- createUnknownJobResult(jobID)))
- .setClusterShutdownFunction(
- status -> {
- shutdownCalled.set(true);
- return CompletableFuture.completedFuture(Acknowledge.get());
- });
-
- final TestingDispatcherGateway dispatcherGateway = dispatcherBuilder.build();
- final CompletableFuture errorHandlerFuture = new CompletableFuture<>();
- final ApplicationDispatcherBootstrap bootstrap =
- createApplicationDispatcherBootstrap(
- 3,
- dispatcherGateway,
- scheduledExecutor,
- errorHandlerFuture::completeExceptionally);
-
- // check that bootstrap shutdown completes exceptionally
- assertException(
- bootstrap.getApplicationCompletionFuture(), UnsuccessfulExecutionException.class);
- // and exception gets propagated to error handler
- assertException(
- bootstrap.getApplicationCompletionFuture(), UnsuccessfulExecutionException.class);
- // and cluster didn't shut down
- assertThat(shutdownCalled.get()).isFalse();
- }
-
- @Test
- void testDuplicateJobSubmissionWithTerminatedJobId() throws Throwable {
- final JobID testJobID = new JobID(0, 2);
- final Configuration configurationUnderTest = getConfiguration();
- configurationUnderTest.set(
- PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID, testJobID.toHexString());
- configurationUnderTest.set(
- HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name());
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- finishedJobGatewayBuilder()
- .setSubmitFunction(
- jobGraph ->
- FutureUtils.completedExceptionally(
- DuplicateJobSubmissionException
- .ofGloballyTerminated(testJobID)));
- final CompletableFuture applicationFuture =
- runApplication(dispatcherBuilder, configurationUnderTest, 1);
- applicationFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
- }
-
- /**
- * In this scenario, job result is no longer present in the {@link
- * org.apache.flink.runtime.dispatcher.Dispatcher dispatcher} (job has terminated and job
- * manager failed over), but we know that job has already terminated from {@link
- * org.apache.flink.runtime.highavailability.JobResultStore}.
- */
- @Test
- void testDuplicateJobSubmissionWithTerminatedJobIdWithUnknownResult() throws Throwable {
- final JobID testJobID = new JobID(0, 2);
- final Configuration configurationUnderTest = getConfiguration();
- configurationUnderTest.set(
- PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID, testJobID.toHexString());
- configurationUnderTest.set(
- HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name());
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- TestingDispatcherGateway.newBuilder()
- .setSubmitFunction(
- jobGraph ->
- FutureUtils.completedExceptionally(
- DuplicateJobSubmissionException
- .ofGloballyTerminated(testJobID)))
- .setRequestJobStatusFunction(
- jobId ->
- FutureUtils.completedExceptionally(
- new FlinkJobNotFoundException(jobId)))
- .setRequestJobResultFunction(
- jobId ->
- FutureUtils.completedExceptionally(
- new FlinkJobNotFoundException(jobId)));
- final CompletableFuture applicationFuture =
- runApplication(dispatcherBuilder, configurationUnderTest, 1);
- applicationFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
- }
-
- /**
- * In this scenario, job result is no longer present in the {@link
- * org.apache.flink.runtime.dispatcher.Dispatcher dispatcher} (job has terminated and job
- * manager failed over), but we know that job has already terminated from {@link
- * org.apache.flink.runtime.highavailability.JobResultStore}.
- */
- @Test
- void testDuplicateJobSubmissionWithTerminatedJobIdWithUnknownResultAttached() throws Throwable {
- final JobID testJobID = new JobID(0, 2);
- final Configuration configurationUnderTest = getConfiguration();
- configurationUnderTest.set(
- PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID, testJobID.toHexString());
- configurationUnderTest.set(
- HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name());
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- TestingDispatcherGateway.newBuilder()
- .setSubmitFunction(
- jobGraph ->
- FutureUtils.completedExceptionally(
- DuplicateJobSubmissionException
- .ofGloballyTerminated(testJobID)))
- .setRequestJobStatusFunction(
- jobId ->
- FutureUtils.completedExceptionally(
- new FlinkJobNotFoundException(jobId)))
- .setRequestJobResultFunction(
- jobId ->
- FutureUtils.completedExceptionally(
- new FlinkJobNotFoundException(jobId)));
- final CompletableFuture applicationFuture =
- runApplication(dispatcherBuilder, configurationUnderTest, 1);
- applicationFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
- }
-
- @Test
- void testDuplicateJobSubmissionWithRunningJobId() throws Throwable {
- final JobID testJobID = new JobID(0, 2);
- final Configuration configurationUnderTest = getConfiguration();
- configurationUnderTest.set(
- PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID, testJobID.toHexString());
- configurationUnderTest.set(
- HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name());
- final TestingDispatcherGateway.Builder dispatcherBuilder =
- TestingDispatcherGateway.newBuilder()
- .setSubmitFunction(
- jobGraph ->
- FutureUtils.completedExceptionally(
- DuplicateJobSubmissionException.of(testJobID)));
- final CompletableFuture applicationFuture =
- runApplication(dispatcherBuilder, configurationUnderTest, 1);
- final ExecutionException executionException =
- assertThrows(
- ExecutionException.class,
- () -> applicationFuture.get(TIMEOUT_SECONDS, TimeUnit.SECONDS));
- final Optional maybeDuplicate =
- ExceptionUtils.findThrowable(
- executionException, DuplicateJobSubmissionException.class);
- assertThat(maybeDuplicate).isPresent();
- assertThat(maybeDuplicate.get().isGloballyTerminated()).isFalse();
- }
-
- @ParameterizedTest
- @EnumSource(
- value = JobStatus.class,
- names = {"FINISHED", "CANCELED", "FAILED"})
- void testShutdownDisabled(JobStatus jobStatus) throws Exception {
- final Configuration configurationUnderTest = getConfiguration();
- configurationUnderTest.set(DeploymentOptions.SHUTDOWN_ON_APPLICATION_FINISH, false);
-
- final TestingDispatcherGateway dispatcherGateway =
- dispatcherGatewayBuilder(jobStatus)
- .setClusterShutdownFunction(
- status -> {
- fail("Cluster shutdown should not be called");
- return CompletableFuture.completedFuture(Acknowledge.get());
- })
- .build();
-
- ApplicationDispatcherBootstrap bootstrap =
- createApplicationDispatcherBootstrap(
- configurationUnderTest, dispatcherGateway, scheduledExecutor);
-
- // Wait until bootstrap is finished to make sure cluster shutdown isn't called
- bootstrap.getBootstrapCompletionFuture().get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
- }
-
- @Test
- void testSubmitFailedJobOnApplicationErrorInHASetup() throws Exception {
- final Configuration configuration = getConfiguration();
- final JobID jobId = new JobID();
- configuration.set(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name());
- configuration.set(DeploymentOptions.SUBMIT_FAILED_JOB_ON_APPLICATION_ERROR, true);
- configuration.set(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID, jobId.toHexString());
- testSubmitFailedJobOnApplicationError(
- configuration,
- (id, t) -> {
- assertThat(id).isEqualTo(jobId);
- assertThat(t)
- .isInstanceOf(ProgramInvocationException.class)
- .hasRootCauseInstanceOf(RuntimeException.class)
- .hasRootCauseMessage(FailingJob.EXCEPTION_MESSAGE);
- });
- }
-
- @Test
- void testSubmitFailedJobOnApplicationErrorInHASetupWithCustomFixedJobId() throws Exception {
- final Configuration configuration = getConfiguration();
- final JobID customFixedJobId = new JobID();
- configuration.set(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name());
- configuration.set(DeploymentOptions.SUBMIT_FAILED_JOB_ON_APPLICATION_ERROR, true);
- configuration.set(
- PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID, customFixedJobId.toHexString());
- testSubmitFailedJobOnApplicationError(
- configuration,
- (jobId, t) -> {
- assertThat(jobId).isEqualTo(customFixedJobId);
- assertThat(t)
- .isInstanceOf(ProgramInvocationException.class)
- .hasRootCauseInstanceOf(RuntimeException.class)
- .hasRootCauseMessage(FailingJob.EXCEPTION_MESSAGE);
- });
- }
-
- private void testSubmitFailedJobOnApplicationError(
- Configuration configuration, BiConsumer failedJobAssertion)
- throws Exception {
- final CompletableFuture submitted = new CompletableFuture<>();
- final TestingDispatcherGateway dispatcherGateway =
- TestingDispatcherGateway.newBuilder()
- .setSubmitFailedFunction(
- (jobId, jobName, t) -> {
- try {
- failedJobAssertion.accept(jobId, t);
- submitted.complete(null);
- return CompletableFuture.completedFuture(Acknowledge.get());
- } catch (Throwable assertion) {
- submitted.completeExceptionally(assertion);
- return FutureUtils.completedExceptionally(assertion);
- }
- })
- .setRequestJobStatusFunction(
- jobId -> submitted.thenApply(ignored -> JobStatus.FAILED))
- .setRequestJobResultFunction(
- jobId ->
- submitted.thenApply(
- ignored ->
- createJobResult(jobId, JobStatus.FAILED)))
- .build();
-
- final ApplicationDispatcherBootstrap bootstrap =
- new ApplicationDispatcherBootstrap(
- FailingJob.getProgram(),
- Collections.emptyList(),
- configuration,
- dispatcherGateway,
- scheduledExecutor,
- exception -> {});
-
- bootstrap.getBootstrapCompletionFuture().get();
- }
-
- @Test
- void testSubmitFailedJobOnApplicationErrorInNonHASetup() throws Exception {
- final Configuration configuration = getConfiguration();
- configuration.set(DeploymentOptions.SUBMIT_FAILED_JOB_ON_APPLICATION_ERROR, true);
- final ApplicationDispatcherBootstrap bootstrap =
- new ApplicationDispatcherBootstrap(
- FailingJob.getProgram(),
- Collections.emptyList(),
- configuration,
- TestingDispatcherGateway.newBuilder().build(),
- scheduledExecutor,
- exception -> {});
- assertThatFuture(bootstrap.getBootstrapCompletionFuture())
- .eventuallyFailsWith(ExecutionException.class)
- .extracting(Throwable::getCause)
- .satisfies(
- e ->
- assertThat(e)
- .isInstanceOf(ApplicationExecutionException.class)
- .hasMessageContaining(
- DeploymentOptions
- .SUBMIT_FAILED_JOB_ON_APPLICATION_ERROR
- .key()));
- }
-
- private TestingDispatcherGateway.Builder finishedJobGatewayBuilder() {
- return dispatcherGatewayBuilder(JobStatus.FINISHED);
- }
-
- private TestingDispatcherGateway.Builder failedJobGatewayBuilder() {
- return dispatcherGatewayBuilder(JobStatus.FAILED);
- }
-
- private TestingDispatcherGateway.Builder canceledJobGatewayBuilder() {
- return dispatcherGatewayBuilder(JobStatus.CANCELED);
- }
-
- private TestingDispatcherGateway.Builder runningJobGatewayBuilder() {
- return dispatcherGatewayBuilder(JobStatus.RUNNING);
- }
-
- private TestingDispatcherGateway.Builder dispatcherGatewayBuilder(JobStatus jobStatus) {
- TestingDispatcherGateway.Builder builder =
- TestingDispatcherGateway.newBuilder()
- .setSubmitFunction(
- jobGraph -> CompletableFuture.completedFuture(Acknowledge.get()))
- .setRequestJobStatusFunction(
- jobId -> CompletableFuture.completedFuture(jobStatus));
- if (jobStatus != JobStatus.RUNNING) {
- builder.setRequestJobResultFunction(
- jobID -> CompletableFuture.completedFuture(createJobResult(jobID, jobStatus)));
- }
- return builder;
- }
-
- private CompletableFuture runApplication(
- TestingDispatcherGateway.Builder dispatcherBuilder, int noOfJobs)
- throws FlinkException {
-
- return runApplication(dispatcherBuilder, getConfiguration(), noOfJobs);
- }
-
- private CompletableFuture runApplication(
- final Configuration configuration, final int noOfJobs) throws Throwable {
-
- final TestingDispatcherGateway.Builder dispatcherBuilder = finishedJobGatewayBuilder();
-
- return runApplication(dispatcherBuilder, configuration, noOfJobs);
- }
-
- private CompletableFuture runApplication(
- TestingDispatcherGateway.Builder dispatcherBuilder,
- Configuration configuration,
- int noOfJobs)
- throws FlinkException {
-
- final PackagedProgram program = getProgram(noOfJobs);
-
- final ApplicationDispatcherBootstrap bootstrap =
- new ApplicationDispatcherBootstrap(
- program,
- Collections.emptyList(),
- configuration,
- dispatcherBuilder.build(),
- scheduledExecutor,
- exception -> {});
-
- return bootstrap.getApplicationCompletionFuture();
- }
-
- private ApplicationDispatcherBootstrap createApplicationDispatcherBootstrap(
- final int noOfJobs,
- final DispatcherGateway dispatcherGateway,
- final ScheduledExecutor scheduledExecutor)
- throws FlinkException {
- return createApplicationDispatcherBootstrap(
- noOfJobs, dispatcherGateway, scheduledExecutor, exception -> {});
- }
-
- private ApplicationDispatcherBootstrap createApplicationDispatcherBootstrap(
- final int noOfJobs,
- final DispatcherGateway dispatcherGateway,
- final ScheduledExecutor scheduledExecutor,
- final FatalErrorHandler errorHandler)
- throws FlinkException {
- return createApplicationDispatcherBootstrap(
- noOfJobs, getConfiguration(), dispatcherGateway, scheduledExecutor, errorHandler);
- }
-
- private ApplicationDispatcherBootstrap createApplicationDispatcherBootstrap(
- final Configuration configuration,
- final DispatcherGateway dispatcherGateway,
- final ScheduledExecutor scheduledExecutor)
- throws FlinkException {
- return createApplicationDispatcherBootstrap(
- 1, configuration, dispatcherGateway, scheduledExecutor, exception -> {});
- }
-
- private ApplicationDispatcherBootstrap createApplicationDispatcherBootstrap(
- final int noOfJobs,
- final Configuration configuration,
- final DispatcherGateway dispatcherGateway,
- final ScheduledExecutor scheduledExecutor,
- final FatalErrorHandler errorHandler)
- throws FlinkException {
- final PackagedProgram program = getProgram(noOfJobs);
- return new ApplicationDispatcherBootstrap(
- program,
- Collections.emptyList(),
- configuration,
- dispatcherGateway,
- scheduledExecutor,
- errorHandler);
- }
-
- private PackagedProgram getProgram(int noOfJobs) throws FlinkException {
- return MultiExecuteJob.getProgram(noOfJobs, true);
- }
-
- private static JobResult createFailedJobResult(final JobID jobId) {
- return createJobResult(jobId, JobStatus.FAILED);
- }
-
- private static JobResult createUnknownJobResult(final JobID jobId) {
- return createJobResult(jobId, null);
- }
-
- private static JobResult createJobResult(
- final JobID jobID, @Nullable final JobStatus jobStatus) {
- JobResult.Builder builder =
- new JobResult.Builder().jobId(jobID).netRuntime(2L).jobStatus(jobStatus);
- if (jobStatus == JobStatus.CANCELED) {
- builder.serializedThrowable(
- new SerializedThrowable(new JobCancellationException(jobID, "Hello", null)));
- } else if (jobStatus == JobStatus.FAILED || jobStatus == null) {
- builder.serializedThrowable(
- new SerializedThrowable(new JobExecutionException(jobID, "bla bla bla")));
- }
- return builder.build();
- }
-
- private static E assertException(
- CompletableFuture future, Class exceptionClass) throws Exception {
-
- try {
- future.get(TIMEOUT_SECONDS, TimeUnit.SECONDS);
- } catch (Throwable e) {
- Optional maybeException = ExceptionUtils.findThrowable(e, exceptionClass);
- if (!maybeException.isPresent()) {
- throw e;
- }
- return maybeException.get();
- }
- throw new Exception(
- "Future should have completed exceptionally with "
- + exceptionClass.getCanonicalName()
- + ".");
- }
-
- private Configuration getConfiguration() {
- final Configuration configuration = new Configuration();
- configuration.set(DeploymentOptions.TARGET, EmbeddedExecutor.NAME);
- return configuration;
- }
-}
diff --git a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationJobUtilsTest.java b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationJobUtilsTest.java
new file mode 100644
index 0000000000000..ef9982b91f938
--- /dev/null
+++ b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationJobUtilsTest.java
@@ -0,0 +1,174 @@
+/*
+ * 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.JobID;
+import org.apache.flink.configuration.ApplicationOptionsInternal;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.HighAvailabilityOptions;
+import org.apache.flink.configuration.PipelineOptionsInternal;
+import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
+import org.apache.flink.util.AbstractID;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import javax.annotation.Nullable;
+
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** Tests for {@link ApplicationJobUtils}. */
+public class ApplicationJobUtilsTest {
+
+ private static final String TEST_HA_CLUSTER_ID = "cluster";
+ private static final String TEST_APPLICATION_ID = "ca0eb040022fbccd4cf05d1e274ae25e";
+ private static final String TEST_JOB_ID = "e79b6d171acd4baa6f421e3631168810";
+
+ private Configuration configuration;
+
+ @BeforeEach
+ void setUp() {
+ configuration = new Configuration();
+ }
+
+ @ParameterizedTest
+ @MethodSource("provideParametersForMaybeFixIds")
+ void testMaybeFixIds(
+ boolean isHAEnabled,
+ boolean isHaClusterIdSet,
+ boolean isApplicationIdSet,
+ boolean isJobIdSet,
+ @Nullable String expectedApplicationId,
+ @Nullable String expectedJobId) {
+ if (isHAEnabled) {
+ configuration.set(
+ HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name());
+ }
+ if (isHaClusterIdSet) {
+ configuration.set(HighAvailabilityOptions.HA_CLUSTER_ID, TEST_HA_CLUSTER_ID);
+ }
+ if (isApplicationIdSet) {
+ configuration.set(ApplicationOptionsInternal.FIXED_APPLICATION_ID, TEST_APPLICATION_ID);
+ }
+ if (isJobIdSet) {
+ configuration.set(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID, TEST_JOB_ID);
+ }
+
+ ApplicationJobUtils.maybeFixIds(configuration);
+
+ assertEquals(
+ expectedApplicationId,
+ configuration.get(ApplicationOptionsInternal.FIXED_APPLICATION_ID));
+
+ assertEquals(
+ expectedJobId, configuration.get(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID));
+ }
+
+ private static Stream provideParametersForMaybeFixIds() {
+ // all combinations for the five input: (isHAEnabled, isHaClusterIdSet, isApplicationIdSet,
+ // isJobIdSet)
+ return Stream.of(
+ Arguments.of(false, false, false, false, null, null),
+ Arguments.of(false, false, false, true, null, TEST_JOB_ID),
+ Arguments.of(false, false, true, false, TEST_APPLICATION_ID, null),
+ Arguments.of(false, false, true, true, TEST_APPLICATION_ID, TEST_JOB_ID),
+ Arguments.of(false, true, false, false, null, null),
+ Arguments.of(false, true, false, true, null, TEST_JOB_ID),
+ Arguments.of(false, true, true, false, TEST_APPLICATION_ID, null),
+ Arguments.of(false, true, true, true, TEST_APPLICATION_ID, TEST_JOB_ID),
+ Arguments.of(
+ true,
+ false,
+ false,
+ false,
+ getAbstractIdFromString(
+ HighAvailabilityOptions.HA_CLUSTER_ID.defaultValue()),
+ getAbstractIdFromString(
+ HighAvailabilityOptions.HA_CLUSTER_ID.defaultValue())),
+ Arguments.of(
+ true,
+ false,
+ false,
+ true,
+ getAbstractIdFromString(
+ HighAvailabilityOptions.HA_CLUSTER_ID.defaultValue()),
+ TEST_JOB_ID),
+ Arguments.of(true, false, true, false, TEST_APPLICATION_ID, TEST_APPLICATION_ID),
+ Arguments.of(true, false, true, true, TEST_APPLICATION_ID, TEST_JOB_ID),
+ Arguments.of(
+ true,
+ true,
+ false,
+ false,
+ getAbstractIdFromString(TEST_HA_CLUSTER_ID),
+ getAbstractIdFromString(TEST_HA_CLUSTER_ID)),
+ Arguments.of(
+ true,
+ true,
+ false,
+ true,
+ getAbstractIdFromString(TEST_HA_CLUSTER_ID),
+ TEST_JOB_ID),
+ Arguments.of(true, true, true, false, TEST_APPLICATION_ID, TEST_APPLICATION_ID),
+ Arguments.of(true, true, true, true, TEST_APPLICATION_ID, TEST_JOB_ID));
+ }
+
+ private static String getAbstractIdFromString(String str) {
+ return (new AbstractID(str.hashCode(), 0)).toHexString();
+ }
+
+ @Test
+ void testAllowExecuteMultipleJobs_HADisabled_NoFixedJobId() {
+ assertEquals(
+ HighAvailabilityMode.NONE.name(),
+ configuration.get(HighAvailabilityOptions.HA_MODE));
+ assertNull(configuration.get(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID));
+
+ assertTrue(ApplicationJobUtils.allowExecuteMultipleJobs(configuration));
+ }
+
+ @Test
+ void testAllowExecuteMultipleJobs_HAEnabled_NoFixedJobId() {
+ final String clusterId = "cluster";
+ configuration.set(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name());
+ configuration.set(HighAvailabilityOptions.HA_CLUSTER_ID, clusterId);
+ assertNull(configuration.get(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID));
+
+ assertFalse(ApplicationJobUtils.allowExecuteMultipleJobs(configuration));
+ }
+
+ @Test
+ void testAllowExecuteMultipleJobs_HAEnabled_FixedJobIdSet() {
+ final String clusterId = "cluster";
+ final JobID testJobID = new JobID(0, 2);
+ configuration.set(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name());
+ configuration.set(HighAvailabilityOptions.HA_CLUSTER_ID, clusterId);
+ configuration.set(PipelineOptionsInternal.PIPELINE_FIXED_JOB_ID, testJobID.toHexString());
+
+ assertFalse(ApplicationJobUtils.allowExecuteMultipleJobs(configuration));
+ }
+}
diff --git a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/PackagedProgramApplicationITCase.java
similarity index 98%
rename from flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java
rename to flink-clients/src/test/java/org/apache/flink/client/deployment/application/PackagedProgramApplicationITCase.java
index b5fb78bf94de1..03570d120951a 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/PackagedProgramApplicationITCase.java
@@ -69,8 +69,8 @@
import static org.assertj.core.api.Assertions.assertThat;
-/** Integration tests related to {@link ApplicationDispatcherBootstrap}. */
-class ApplicationDispatcherBootstrapITCase {
+/** Integration tests related to {@link PackagedProgramApplication}. */
+class PackagedProgramApplicationITCase {
@RegisterExtension
static final TestExecutorExtension EXECUTOR_EXTENSION =
@@ -238,8 +238,7 @@ void testSubmitFailedJobOnApplicationError() throws Exception {
final ArchivedExecutionGraph graph = cluster.getArchivedExecutionGraph(jobId).get();
assertThat(graph.getJobID()).isEqualTo(jobId);
- assertThat(graph.getJobName())
- .isEqualTo(ApplicationDispatcherBootstrap.FAILED_JOB_NAME);
+ assertThat(graph.getJobName()).isEqualTo(PackagedProgramApplication.FAILED_JOB_NAME);
assertThat(graph.getFailureInfo())
.isNotNull()
.extracting(ErrorInfo::getException)
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ApplicationOptionsInternal.java b/flink-core/src/main/java/org/apache/flink/configuration/ApplicationOptionsInternal.java
new file mode 100644
index 0000000000000..7191a21e68374
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ApplicationOptionsInternal.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.configuration;
+
+import static org.apache.flink.configuration.ConfigOptions.key;
+
+/** Application options that are not meant to be used by the user. */
+public class ApplicationOptionsInternal {
+ public static final ConfigOption FIXED_APPLICATION_ID =
+ key("$internal.application.id")
+ .stringType()
+ .noDefaultValue()
+ .withDescription(
+ "**DO NOT USE** The static ApplicationId to be used for the application. "
+ + "For fault-tolerance, this value needs to stay the same across runs.");
+}
diff --git a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot
index b1b5497354e05..1444b47b68481 100644
--- a/flink-runtime-web/src/test/resources/rest_api_v1.snapshot
+++ b/flink-runtime-web/src/test/resources/rest_api_v1.snapshot
@@ -1,5 +1,167 @@
{
"calls" : [ {
+ "url" : "/applications/overview",
+ "method" : "GET",
+ "status-code" : "200 OK",
+ "file-upload" : false,
+ "path-parameters" : {
+ "pathParameters" : [ ]
+ },
+ "query-parameters" : {
+ "queryParameters" : [ ]
+ },
+ "request" : {
+ "type" : "object",
+ "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:EmptyRequestBody"
+ },
+ "response" : {
+ "type" : "object",
+ "id" : "urn:jsonschema:org:apache:flink:runtime:messages:webmonitor:MultipleApplicationsDetails",
+ "properties" : {
+ "applications" : {
+ "type" : "array",
+ "items" : {
+ "type" : "object",
+ "id" : "urn:jsonschema:org:apache:flink:runtime:messages:webmonitor:ApplicationDetails",
+ "properties" : {
+ "id" : {
+ "type" : "any"
+ },
+ "name" : {
+ "type" : "string"
+ },
+ "start-time" : {
+ "type" : "integer"
+ },
+ "end-time" : {
+ "type" : "integer"
+ },
+ "duration" : {
+ "type" : "integer"
+ },
+ "status" : {
+ "type" : "string"
+ },
+ "jobs" : {
+ "type" : "object",
+ "additionalProperties" : {
+ "type" : "integer"
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+ }, {
+ "url" : "/applications/:applicationid",
+ "method" : "GET",
+ "status-code" : "200 OK",
+ "file-upload" : false,
+ "path-parameters" : {
+ "pathParameters" : [ {
+ "key" : "applicationid"
+ } ]
+ },
+ "query-parameters" : {
+ "queryParameters" : [ ]
+ },
+ "request" : {
+ "type" : "object",
+ "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:EmptyRequestBody"
+ },
+ "response" : {
+ "type" : "object",
+ "id" : "urn:jsonschema:org:apache:flink:runtime:messages:webmonitor:ApplicationDetailsInfo",
+ "properties" : {
+ "id" : {
+ "type" : "any"
+ },
+ "name" : {
+ "type" : "string"
+ },
+ "status" : {
+ "type" : "string"
+ },
+ "start-time" : {
+ "type" : "integer"
+ },
+ "end-time" : {
+ "type" : "integer"
+ },
+ "duration" : {
+ "type" : "integer"
+ },
+ "timestamps" : {
+ "type" : "object",
+ "additionalProperties" : {
+ "type" : "integer"
+ }
+ },
+ "jobs" : {
+ "type" : "array",
+ "items" : {
+ "type" : "object",
+ "id" : "urn:jsonschema:org:apache:flink:runtime:messages:webmonitor:JobDetails",
+ "properties" : {
+ "jid" : {
+ "type" : "any"
+ },
+ "name" : {
+ "type" : "string"
+ },
+ "start-time" : {
+ "type" : "integer"
+ },
+ "end-time" : {
+ "type" : "integer"
+ },
+ "duration" : {
+ "type" : "integer"
+ },
+ "state" : {
+ "type" : "string",
+ "enum" : [ "INITIALIZING", "CREATED", "RUNNING", "FAILING", "FAILED", "CANCELLING", "CANCELED", "FINISHED", "RESTARTING", "SUSPENDED", "RECONCILING" ]
+ },
+ "last-modification" : {
+ "type" : "integer"
+ },
+ "tasks" : {
+ "type" : "object",
+ "additionalProperties" : {
+ "type" : "integer"
+ }
+ },
+ "pending-operators" : {
+ "type" : "integer"
+ }
+ }
+ }
+ }
+ }
+ }
+ }, {
+ "url" : "/applications/:applicationid/cancel",
+ "method" : "POST",
+ "status-code" : "202 Accepted",
+ "file-upload" : false,
+ "path-parameters" : {
+ "pathParameters" : [ {
+ "key" : "applicationid"
+ } ]
+ },
+ "query-parameters" : {
+ "queryParameters" : [ ]
+ },
+ "request" : {
+ "type" : "object",
+ "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:EmptyRequestBody"
+ },
+ "response" : {
+ "type" : "object",
+ "id" : "urn:jsonschema:org:apache:flink:runtime:rest:messages:EmptyResponseBody"
+ }
+ }, {
"url" : "/cluster",
"method" : "DELETE",
"status-code" : "200 OK",
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/DuplicateApplicationSubmissionException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/DuplicateApplicationSubmissionException.java
new file mode 100644
index 0000000000000..f5b0ecc9dd45c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/DuplicateApplicationSubmissionException.java
@@ -0,0 +1,38 @@
+/*
+ * 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.client;
+
+import org.apache.flink.api.common.ApplicationID;
+import org.apache.flink.util.FlinkException;
+
+public class DuplicateApplicationSubmissionException extends FlinkException {
+
+ private static final long serialVersionUID = 2818087325120827524L;
+
+ private final ApplicationID applicationId;
+
+ public DuplicateApplicationSubmissionException(ApplicationID applicationId) {
+ super("Application has already been submitted.");
+ this.applicationId = applicationId;
+ }
+
+ public ApplicationID getApplicationId() {
+ return applicationId;
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/ApplicationBootstrap.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/ApplicationBootstrap.java
new file mode 100644
index 0000000000000..f781dedff1ea5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/ApplicationBootstrap.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.dispatcher;
+
+import org.apache.flink.runtime.application.AbstractApplication;
+
+/**
+ * A {@link DispatcherBootstrap} which wraps an {@link AbstractApplication} for execution upon
+ * dispatcher initialization.
+ */
+public class ApplicationBootstrap implements DispatcherBootstrap {
+ private final AbstractApplication application;
+
+ public ApplicationBootstrap(AbstractApplication application) {
+ this.application = application;
+ }
+
+ @Override
+ public void stop() throws Exception {
+ application.cancel();
+ }
+
+ public AbstractApplication getApplication() {
+ return application;
+ }
+}
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 63e10ec5cb7d8..b766159dc73ce 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
@@ -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.JobID;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.api.java.tuple.Tuple2;
@@ -36,10 +38,12 @@
import org.apache.flink.core.execution.SavepointFormatType;
import org.apache.flink.core.failure.FailureEnricher;
import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.application.AbstractApplication;
import org.apache.flink.runtime.blob.BlobServer;
import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
import org.apache.flink.runtime.checkpoint.Checkpoints;
import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
+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;
@@ -69,11 +73,16 @@
import org.apache.flink.runtime.jobmaster.JobResult;
import org.apache.flink.runtime.jobmaster.factories.DefaultJobManagerJobMetricGroupFactory;
import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.FlinkApplicationNotFoundException;
+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.ApplicationDetails;
+import org.apache.flink.runtime.messages.webmonitor.ApplicationDetailsInfo;
import org.apache.flink.runtime.messages.webmonitor.ClusterOverview;
import org.apache.flink.runtime.messages.webmonitor.JobDetails;
import org.apache.flink.runtime.messages.webmonitor.JobsOverview;
+import org.apache.flink.runtime.messages.webmonitor.MultipleApplicationsDetails;
import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
import org.apache.flink.runtime.metrics.MetricNames;
import org.apache.flink.runtime.metrics.groups.JobManagerMetricGroup;
@@ -133,6 +142,7 @@
import java.util.stream.Collectors;
import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
/**
* Base class for the Dispatcher component. The Dispatcher component is responsible for receiving
@@ -212,6 +222,10 @@ public abstract class Dispatcher extends FencedRpcEndpoint
*/
private final Set pendingJobResourceRequirementsUpdates = new HashSet<>();
+ private final Map applications = new HashMap<>();
+
+ private final Map> recoveredApplicationJobIds = new HashMap<>();
+
/** Enum to distinguish between initial job submission and re-submission for recovery. */
protected enum ExecutionType {
SUBMISSION,
@@ -366,6 +380,10 @@ public void onStart() throws Exception {
getSelfGateway(DispatcherGateway.class),
this.getRpcService().getScheduledExecutor(),
this::onFatalError);
+
+ if (dispatcherBootstrap instanceof ApplicationBootstrap) {
+ submitApplication(((ApplicationBootstrap) dispatcherBootstrap).getApplication()).get();
+ }
}
private void startDispatcherServices() throws Exception {
@@ -410,6 +428,13 @@ private void startRecoveredJobs() {
private void runRecoveredJob(final ExecutionPlan recoveredJob) {
checkNotNull(recoveredJob);
+ if (recoveredJob.getApplicationId().isPresent()) {
+ recoveredApplicationJobIds
+ .computeIfAbsent(
+ recoveredJob.getApplicationId().get(), ignored -> new HashSet<>())
+ .add(recoveredJob.getJobID());
+ }
+
initJobClientExpiredTime(recoveredJob);
try (MdcCloseable ignored =
@@ -497,6 +522,10 @@ public CompletableFuture onStop() {
() -> {
dispatcherBootstrap.stop();
+ for (AbstractApplication application : applications.values()) {
+ application.dispose();
+ }
+
stopDispatcherServices();
log.info("Stopped dispatcher {}.", getAddress());
@@ -579,6 +608,34 @@ public CompletableFuture submitFailedJob(
return archiveExecutionGraphToHistoryServer(executionGraphInfo);
}
+ /** This method must be called from the main thread. */
+ private CompletableFuture submitApplication(AbstractApplication application) {
+ final ApplicationID applicationId = application.getApplicationId();
+ log.info(
+ "Received application submission '{}' ({}).", application.getName(), applicationId);
+
+ if (applications.containsKey(applicationId)) {
+ log.warn("Application with id {} already exists.", applicationId);
+ throw new CompletionException(
+ new DuplicateApplicationSubmissionException(applicationId));
+ }
+ applications.put(applicationId, application);
+ Set jobs = recoveredApplicationJobIds.remove(applicationId);
+ if (jobs != null) {
+ jobs.forEach(application::addJob);
+ }
+ return application.execute(
+ getSelfGateway(DispatcherGateway.class),
+ getRpcService().getScheduledExecutor(),
+ getMainThreadExecutor(),
+ this::onFatalError);
+ }
+
+ @VisibleForTesting
+ Map getApplications() {
+ return applications;
+ }
+
/**
* Checks whether the given job has already been executed.
*
@@ -595,7 +652,28 @@ private CompletableFuture internalSubmitJob(ExecutionPlan execution
applyParallelismOverrides((JobGraph) executionPlan);
}
- log.info("Submitting job '{}' ({}).", executionPlan.getName(), executionPlan.getJobID());
+ final JobID jobId = executionPlan.getJobID();
+ final String jobName = executionPlan.getName();
+
+ if (executionPlan.getApplicationId().isPresent()) {
+ ApplicationID applicationId = executionPlan.getApplicationId().get();
+ log.info(
+ "Submitting job '{}' ({}) with associated application ({}).",
+ jobName,
+ jobId,
+ applicationId);
+ checkState(
+ applications.containsKey(applicationId),
+ "Application %s not found.",
+ applicationId);
+ applications.get(applicationId).addJob(jobId);
+ } else {
+ // TODO update the message after SingleJobApplication is implemented
+ // This can occur in two cases:
+ // 1. CLI/REST submissions of jobs without an application
+ // 2. Tests for submitJob that submit jobs without an application
+ log.info("Submitting job '{}' ({}) without associated application.", jobName, jobId);
+ }
// track as an outstanding job
submittedAndWaitingTerminationJobIDs.add(executionPlan.getJobID());
@@ -649,7 +727,7 @@ private void persistAndRunJob(ExecutionPlan executionPlan) throws Exception {
}
private JobManagerRunner createJobMasterRunner(ExecutionPlan executionPlan) throws Exception {
- Preconditions.checkState(!jobManagerRunnerRegistry.isRegistered(executionPlan.getJobID()));
+ checkState(!jobManagerRunnerRegistry.isRegistered(executionPlan.getJobID()));
return jobManagerRunnerFactory.createJobManagerRunner(
executionPlan,
configuration,
@@ -664,7 +742,7 @@ private JobManagerRunner createJobMasterRunner(ExecutionPlan executionPlan) thro
}
private JobManagerRunner createJobCleanupRunner(JobResult dirtyJobResult) throws Exception {
- Preconditions.checkState(!jobManagerRunnerRegistry.isRegistered(dirtyJobResult.getJobId()));
+ checkState(!jobManagerRunnerRegistry.isRegistered(dirtyJobResult.getJobId()));
return cleanupRunnerFactory.create(
dirtyJobResult,
highAvailabilityServices.getCheckpointRecoveryFactory(),
@@ -684,7 +762,7 @@ private void runJob(JobManagerRunner jobManagerRunner, ExecutionType executionTy
.getResultFuture()
.handleAsync(
(jobManagerRunnerResult, throwable) -> {
- Preconditions.checkState(
+ checkState(
jobManagerRunnerRegistry.isRegistered(jobId)
&& jobManagerRunnerRegistry.get(jobId)
== jobManagerRunner,
@@ -830,6 +908,28 @@ public CompletableFuture cancelJob(JobID jobId, Duration timeout) {
return FutureUtils.completedExceptionally(new FlinkJobNotFoundException(jobId));
}
+ @Override
+ public CompletableFuture cancelApplication(
+ ApplicationID applicationId, Duration timeout) {
+ if (!applications.containsKey(applicationId)) {
+ return FutureUtils.completedExceptionally(
+ new FlinkApplicationNotFoundException(applicationId));
+ }
+ AbstractApplication application = applications.get(applicationId);
+ ApplicationState current = application.getApplicationStatus();
+ if (current.isTerminalState()) {
+ if (current == ApplicationState.CANCELED) {
+ return CompletableFuture.completedFuture(Acknowledge.get());
+ } else {
+ return FutureUtils.completedExceptionally(
+ new FlinkApplicationTerminatedWithoutCancellationException(
+ applicationId, current));
+ }
+ }
+ applications.get(applicationId).cancel();
+ return CompletableFuture.completedFuture(Acknowledge.get());
+ }
+
@Override
public CompletableFuture requestClusterOverview(Duration timeout) {
CompletableFuture taskManagerOverviewFuture =
@@ -895,6 +995,114 @@ public CompletableFuture requestMultipleJobDetails(Duration
});
}
+ @Override
+ public CompletableFuture requestMultipleApplicationDetails(
+ Duration timeout) {
+ List> applicationDetailsFutures =
+ applications.values().stream()
+ .map(application -> createApplicationDetails(application, timeout))
+ .collect(Collectors.toList());
+ return FutureUtils.combineAll(applicationDetailsFutures)
+ .thenCompose(
+ combinedApplicationDetails ->
+ CompletableFuture.completedFuture(
+ new MultipleApplicationsDetails(
+ combinedApplicationDetails)));
+ }
+
+ private CompletableFuture createApplicationDetails(
+ AbstractApplication application, Duration timeout) {
+ ApplicationState applicationStatus = application.getApplicationStatus();
+ long startTime = application.getStatusTimestamp(ApplicationState.RUNNING);
+ long endTime =
+ applicationStatus.isTerminalState()
+ ? application.getStatusTimestamp(applicationStatus)
+ : -1L;
+ long duration = (endTime >= 0L ? endTime : System.currentTimeMillis()) - startTime;
+ Map jobInfo = new HashMap<>();
+ List> individualJobStatus =
+ application.getJobs().stream()
+ .map(jobId -> requestJobStatus(jobId, timeout))
+ .collect(Collectors.toList());
+ CompletableFuture> combinedJobStatus =
+ FutureUtils.combineAll(individualJobStatus);
+ return combinedJobStatus.thenCompose(
+ jobStatuses -> {
+ for (JobStatus status : jobStatuses) {
+ if (status != null) {
+ jobInfo.compute(
+ status.name(),
+ (key, oldValue) -> (oldValue == null ? 1 : oldValue + 1));
+ }
+ }
+ return CompletableFuture.completedFuture(
+ new ApplicationDetails(
+ application.getApplicationId(),
+ application.getName(),
+ startTime,
+ endTime,
+ duration,
+ applicationStatus.toString(),
+ jobInfo));
+ });
+ }
+
+ @Override
+ public CompletableFuture requestApplication(
+ ApplicationID applicationId, Duration timeout) {
+ if (!applications.containsKey(applicationId)) {
+ return FutureUtils.completedExceptionally(
+ new FlinkApplicationNotFoundException(applicationId));
+ }
+ AbstractApplication application = applications.get(applicationId);
+ ApplicationState applicationStatus = application.getApplicationStatus();
+ long startTime = application.getStatusTimestamp(ApplicationState.RUNNING);
+ long endTime =
+ applicationStatus.isTerminalState()
+ ? application.getStatusTimestamp(applicationStatus)
+ : -1L;
+ long duration = (endTime >= 0L ? endTime : System.currentTimeMillis()) - startTime;
+ final Map timestamps =
+ CollectionUtil.newHashMapWithExpectedSize(ApplicationState.values().length);
+ for (ApplicationState status : ApplicationState.values()) {
+ timestamps.put(status.toString(), application.getStatusTimestamp(status));
+ }
+ List> jobDetailsFutures =
+ application.getJobs().stream()
+ .map(jobId -> requestJobDetails(jobId, timeout))
+ .collect(Collectors.toList());
+ return FutureUtils.combineAll(jobDetailsFutures)
+ .thenCompose(
+ combinedJobDetails ->
+ CompletableFuture.completedFuture(
+ new ApplicationDetailsInfo(
+ application.getApplicationId(),
+ application.getName(),
+ applicationStatus.toString(),
+ startTime,
+ endTime,
+ duration,
+ timestamps,
+ combinedJobDetails)));
+ }
+
+ private CompletableFuture requestJobDetails(JobID jobId, Duration timeout) {
+ Optional maybeJob = getJobManagerRunner(jobId);
+ return maybeJob.map(job -> job.requestJobDetails(timeout))
+ .orElseGet(
+ () -> {
+ // is it a completed job?
+ final JobDetails jobDetails =
+ executionGraphInfoStore.getAvailableJobDetails(jobId);
+ if (jobDetails == null) {
+ return FutureUtils.completedExceptionally(
+ new FlinkJobNotFoundException(jobId));
+ } else {
+ return CompletableFuture.completedFuture(jobDetails);
+ }
+ });
+ }
+
@Override
public CompletableFuture requestJobStatus(JobID jobId, Duration timeout) {
Optional maybeJob = getJobManagerRunner(jobId);
@@ -1277,7 +1485,7 @@ private void setClientHeartbeatTimeoutForInitializedJob() {
@VisibleForTesting
void registerJobManagerRunnerTerminationFuture(
JobID jobId, CompletableFuture jobManagerRunnerTerminationFuture) {
- Preconditions.checkState(!jobManagerRunnerTerminationFutures.containsKey(jobId));
+ checkState(!jobManagerRunnerTerminationFutures.containsKey(jobId));
jobManagerRunnerTerminationFutures.put(jobId, jobManagerRunnerTerminationFuture);
// clean up the pending termination future
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
index 551168ea4ef10..3506526d1df05 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
@@ -18,6 +18,7 @@
package org.apache.flink.runtime.jobgraph;
+import org.apache.flink.api.common.ApplicationID;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.JobID;
@@ -49,6 +50,7 @@
import java.util.List;
import java.util.Map;
import java.util.Objects;
+import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
@@ -83,6 +85,9 @@ public class JobGraph implements ExecutionPlan {
/** ID of this job. May be set if specific job id is desired (e.g. session management) */
private JobID jobID;
+ /** ID of the application this job belongs to. */
+ @Nullable private ApplicationID applicationId;
+
/** Name of this job. */
private final String jobName;
@@ -134,20 +139,21 @@ public class JobGraph implements ExecutionPlan {
* @param jobName The name of the job.
*/
public JobGraph(String jobName) {
- this(null, jobName);
+ this(null, null, jobName);
}
/**
* Constructs a new job graph with the given job ID (or a random ID, if {@code null} is passed),
- * the given name and the given execution configuration (see {@link ExecutionConfig}). The
- * ExecutionConfig will be serialized and can't be modified afterwards.
+ * the given application ID, the given name and the given execution configuration (see {@link
+ * ExecutionConfig}). The ExecutionConfig will be serialized and can't be modified afterwards.
*
* @param jobId The id of the job. A random ID is generated, if {@code null} is passed.
* @param jobName The name of the job.
*/
- public JobGraph(@Nullable JobID jobId, String jobName) {
+ public JobGraph(@Nullable JobID jobId, @Nullable ApplicationID applicationId, String jobName) {
this.jobID = jobId == null ? new JobID() : jobId;
this.jobName = jobName == null ? "(unnamed job)" : jobName;
+ this.applicationId = applicationId;
try {
setExecutionConfig(new ExecutionConfig());
@@ -167,7 +173,7 @@ public JobGraph(@Nullable JobID jobId, String jobName) {
* @param vertices The vertices to add to the graph.
*/
public JobGraph(@Nullable JobID jobId, String jobName, JobVertex... vertices) {
- this(jobId, jobName);
+ this(jobId, null, jobName);
for (JobVertex vertex : vertices) {
addVertex(vertex);
@@ -191,6 +197,21 @@ public void setJobID(JobID jobID) {
this.jobID = jobID;
}
+ /**
+ * Returns the ID of the application this job belongs to.
+ *
+ * @return the ID of the application
+ */
+ @Override
+ public Optional getApplicationId() {
+ return Optional.ofNullable(applicationId);
+ }
+
+ /** Sets the ID of the application. */
+ public void setApplicationId(ApplicationID applicationId) {
+ this.applicationId = checkNotNull(applicationId);
+ }
+
/**
* Returns the name assigned to the job graph.
*
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/FlinkApplicationNotFoundException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/FlinkApplicationNotFoundException.java
new file mode 100644
index 0000000000000..2648d05f49571
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/FlinkApplicationNotFoundException.java
@@ -0,0 +1,34 @@
+/*
+ * 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.messages;
+
+import org.apache.flink.api.common.ApplicationID;
+import org.apache.flink.util.FlinkException;
+
+/**
+ * Exception indicating that we could not find a Flink application with the given application ID.
+ */
+public class FlinkApplicationNotFoundException extends FlinkException {
+
+ private static final long serialVersionUID = 2294698055059659025L;
+
+ public FlinkApplicationNotFoundException(ApplicationID applicationId) {
+ super("Could not find Flink application (" + applicationId + ')');
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/FlinkApplicationTerminatedWithoutCancellationException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/FlinkApplicationTerminatedWithoutCancellationException.java
new file mode 100644
index 0000000000000..6ce30f3e00a6f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/FlinkApplicationTerminatedWithoutCancellationException.java
@@ -0,0 +1,53 @@
+/*
+ * 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.messages;
+
+import org.apache.flink.api.common.ApplicationID;
+import org.apache.flink.api.common.ApplicationState;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * Exception indicating that the Flink application with the given application ID has terminated
+ * without cancellation.
+ */
+public class FlinkApplicationTerminatedWithoutCancellationException extends FlinkException {
+
+ private static final long serialVersionUID = 2294698055059659024L;
+
+ private final ApplicationState status;
+
+ public FlinkApplicationTerminatedWithoutCancellationException(
+ ApplicationID applicationId, ApplicationState status) {
+ super(
+ String.format(
+ "Flink application (%s) was not canceled, but instead %s.",
+ applicationId, assertNotCanceled(status)));
+ this.status = status;
+ }
+
+ public ApplicationState getApplicationStatus() {
+ return status;
+ }
+
+ private static ApplicationState assertNotCanceled(ApplicationState status) {
+ Preconditions.checkState(status != ApplicationState.CANCELED);
+ return status;
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/ApplicationDetails.java b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/ApplicationDetails.java
new file mode 100644
index 0000000000000..3f15819638371
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/ApplicationDetails.java
@@ -0,0 +1,170 @@
+/*
+ * 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.messages.webmonitor;
+
+import org.apache.flink.api.common.ApplicationID;
+import org.apache.flink.runtime.rest.messages.json.ApplicationIDDeserializer;
+import org.apache.flink.runtime.rest.messages.json.ApplicationIDSerializer;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+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 java.io.Serializable;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Overview of an application. */
+public class ApplicationDetails implements Serializable {
+
+ private static final long serialVersionUID = -3391462110304948765L;
+
+ private static final String FIELD_NAME_APPLICATION_ID = "id";
+ private static final String FIELD_NAME_APPLICATION_NAME = "name";
+ private static final String FIELD_NAME_START_TIME = "start-time";
+ private static final String FIELD_NAME_END_TIME = "end-time";
+ private static final String FIELD_NAME_DURATION = "duration";
+ private static final String FIELD_NAME_STATUS = "status";
+ private static final String FIELD_NAME_JOBS = "jobs";
+
+ private final ApplicationID applicationId;
+
+ private final String applicationName;
+
+ private final long startTime;
+
+ private final long endTime;
+
+ private final long duration;
+
+ private final String status;
+
+ private final Map jobInfo;
+
+ @JsonCreator
+ public ApplicationDetails(
+ @JsonProperty(FIELD_NAME_APPLICATION_ID)
+ @JsonDeserialize(using = ApplicationIDDeserializer.class)
+ ApplicationID applicationId,
+ @JsonProperty(FIELD_NAME_APPLICATION_NAME) String applicationName,
+ @JsonProperty(FIELD_NAME_START_TIME) long startTime,
+ @JsonProperty(FIELD_NAME_END_TIME) long endTime,
+ @JsonProperty(FIELD_NAME_DURATION) long duration,
+ @JsonProperty(FIELD_NAME_STATUS) String status,
+ @JsonProperty(FIELD_NAME_JOBS) Map jobInfo) {
+ this.applicationId = checkNotNull(applicationId);
+ this.applicationName = checkNotNull(applicationName);
+ this.startTime = startTime;
+ this.endTime = endTime;
+ this.duration = duration;
+ this.status = checkNotNull(status);
+ this.jobInfo = checkNotNull(jobInfo);
+ }
+
+ // ------------------------------------------------------------------------
+
+ @JsonProperty(FIELD_NAME_APPLICATION_ID)
+ @JsonSerialize(using = ApplicationIDSerializer.class)
+ public ApplicationID getApplicationId() {
+ return applicationId;
+ }
+
+ @JsonProperty(FIELD_NAME_APPLICATION_NAME)
+ public String getApplicationName() {
+ return applicationName;
+ }
+
+ @JsonProperty(FIELD_NAME_START_TIME)
+ public long getStartTime() {
+ return startTime;
+ }
+
+ @JsonProperty(FIELD_NAME_END_TIME)
+ public long getEndTime() {
+ return endTime;
+ }
+
+ @JsonProperty(FIELD_NAME_DURATION)
+ public long getDuration() {
+ return duration;
+ }
+
+ @JsonProperty(FIELD_NAME_STATUS)
+ public String getStatus() {
+ return status;
+ }
+
+ @JsonProperty(FIELD_NAME_JOBS)
+ Map getJobInfo() {
+ return jobInfo;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ } else if (o != null && o.getClass() == ApplicationDetails.class) {
+ ApplicationDetails that = (ApplicationDetails) o;
+
+ return this.endTime == that.endTime
+ && this.startTime == that.startTime
+ && this.duration == that.duration
+ && this.status.equals(that.status)
+ && this.applicationId.equals(that.applicationId)
+ && this.applicationName.equals(that.applicationName)
+ && this.jobInfo.equals(that.jobInfo);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ int result = applicationId.hashCode();
+ result = 31 * result + applicationName.hashCode();
+ result = 31 * result + (int) (startTime ^ (startTime >>> 32));
+ result = 31 * result + (int) (endTime ^ (endTime >>> 32));
+ result = 31 * result + (int) (duration ^ (duration >>> 32));
+ result = 31 * result + status.hashCode();
+ result = 31 * result + jobInfo.hashCode();
+ return result;
+ }
+
+ @Override
+ public String toString() {
+ return "ApplicationDetails {"
+ + "applicationId="
+ + applicationId
+ + ", applicationName='"
+ + applicationName
+ + ", startTime="
+ + startTime
+ + ", endTime="
+ + endTime
+ + ", duration="
+ + duration
+ + ", status="
+ + status
+ + ", jobInfo="
+ + jobInfo
+ + '}';
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/ApplicationDetailsInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/ApplicationDetailsInfo.java
new file mode 100644
index 0000000000000..fcf66d03f17f4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/ApplicationDetailsInfo.java
@@ -0,0 +1,195 @@
+/*
+ * 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.messages.webmonitor;
+
+import org.apache.flink.api.common.ApplicationID;
+import org.apache.flink.runtime.rest.messages.ResponseBody;
+import org.apache.flink.runtime.rest.messages.json.ApplicationIDDeserializer;
+import org.apache.flink.runtime.rest.messages.json.ApplicationIDSerializer;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+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 java.io.Serializable;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Objects;
+
+/** Details about an application. */
+public class ApplicationDetailsInfo implements ResponseBody, Serializable {
+
+ private static final long serialVersionUID = -3391462110304948764L;
+
+ public static final String FIELD_NAME_APPLICATION_ID = "id";
+
+ public static final String FIELD_NAME_APPLICATION_NAME = "name";
+
+ public static final String FIELD_NAME_APPLICATION_STATUS = "status";
+
+ private static final String FIELD_NAME_START_TIME = "start-time";
+
+ private static final String FIELD_NAME_END_TIME = "end-time";
+
+ private static final String FIELD_NAME_DURATION = "duration";
+
+ public static final String FIELD_NAME_TIMESTAMPS = "timestamps";
+
+ public static final String FIELD_NAME_JOBS = "jobs";
+
+ private final ApplicationID applicationId;
+
+ private final String applicationName;
+
+ private final String applicationStatus;
+
+ private final long startTime;
+
+ private final long endTime;
+
+ private final long duration;
+
+ private final Map timestamps;
+
+ private final Collection jobs;
+
+ @JsonCreator
+ public ApplicationDetailsInfo(
+ @JsonDeserialize(using = ApplicationIDDeserializer.class)
+ @JsonProperty(FIELD_NAME_APPLICATION_ID)
+ ApplicationID applicationId,
+ @JsonProperty(FIELD_NAME_APPLICATION_NAME) String applicationName,
+ @JsonProperty(FIELD_NAME_APPLICATION_STATUS) String applicationStatus,
+ @JsonProperty(FIELD_NAME_START_TIME) long startTime,
+ @JsonProperty(FIELD_NAME_END_TIME) long endTime,
+ @JsonProperty(FIELD_NAME_DURATION) long duration,
+ @JsonProperty(FIELD_NAME_TIMESTAMPS) Map timestamps,
+ @JsonProperty(FIELD_NAME_JOBS) Collection jobs) {
+ this.applicationId = Preconditions.checkNotNull(applicationId);
+ this.applicationName = Preconditions.checkNotNull(applicationName);
+ this.applicationStatus = Preconditions.checkNotNull(applicationStatus);
+ this.startTime = startTime;
+ this.endTime = endTime;
+ this.duration = duration;
+ this.timestamps = Preconditions.checkNotNull(timestamps);
+ this.jobs = Preconditions.checkNotNull(jobs);
+ }
+
+ // ------------------------------------------------------------------------
+
+ @JsonProperty(FIELD_NAME_APPLICATION_ID)
+ @JsonSerialize(using = ApplicationIDSerializer.class)
+ public ApplicationID getApplicationId() {
+ return applicationId;
+ }
+
+ @JsonProperty(FIELD_NAME_APPLICATION_NAME)
+ public String getApplicationName() {
+ return applicationName;
+ }
+
+ @JsonProperty(FIELD_NAME_APPLICATION_STATUS)
+ public String getApplicationStatus() {
+ return applicationStatus;
+ }
+
+ @JsonProperty(FIELD_NAME_START_TIME)
+ public long getStartTime() {
+ return startTime;
+ }
+
+ @JsonProperty(FIELD_NAME_END_TIME)
+ public long getEndTime() {
+ return endTime;
+ }
+
+ @JsonProperty(FIELD_NAME_DURATION)
+ public long getDuration() {
+ return duration;
+ }
+
+ @JsonProperty(FIELD_NAME_TIMESTAMPS)
+ public Map getTimestamps() {
+ return timestamps;
+ }
+
+ @JsonProperty(FIELD_NAME_JOBS)
+ public Collection getJobs() {
+ return jobs;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ } else if (o != null && o.getClass() == ApplicationDetailsInfo.class) {
+ ApplicationDetailsInfo that = (ApplicationDetailsInfo) o;
+
+ return this.endTime == that.endTime
+ && this.startTime == that.startTime
+ && this.duration == that.duration
+ && this.applicationStatus.equals(that.applicationStatus)
+ && this.applicationId.equals(that.applicationId)
+ && this.applicationName.equals(that.applicationName)
+ && Objects.equals(this.timestamps, that.timestamps)
+ && Objects.equals(this.jobs, that.jobs);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(
+ applicationId,
+ applicationName,
+ applicationStatus,
+ startTime,
+ endTime,
+ duration,
+ timestamps,
+ jobs);
+ }
+
+ @Override
+ public String toString() {
+ return "ApplicationDetailsInfo {"
+ + "applicationId="
+ + applicationId
+ + ", applicationName='"
+ + applicationName
+ + ", startTime="
+ + startTime
+ + ", endTime="
+ + endTime
+ + ", duration="
+ + duration
+ + ", status="
+ + applicationStatus
+ + ", timestamps="
+ + timestamps
+ + ", jobs="
+ + jobs
+ + '}';
+ }
+
+ // ------------------------------------------------------------------------
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/MultipleApplicationsDetails.java b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/MultipleApplicationsDetails.java
new file mode 100644
index 0000000000000..794c311bf991c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/messages/webmonitor/MultipleApplicationsDetails.java
@@ -0,0 +1,76 @@
+/*
+ * 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.messages.webmonitor;
+
+import org.apache.flink.runtime.rest.messages.ResponseBody;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Objects;
+
+/** Overview of all applications. */
+public class MultipleApplicationsDetails implements ResponseBody, Serializable {
+
+ private static final long serialVersionUID = -1526236139616019126L;
+
+ public static final String FIELD_NAME_APPLICATIONS = "applications";
+
+ @JsonProperty(FIELD_NAME_APPLICATIONS)
+ private final Collection applications;
+
+ @JsonCreator
+ public MultipleApplicationsDetails(
+ @JsonProperty(FIELD_NAME_APPLICATIONS) Collection applications) {
+ this.applications = Preconditions.checkNotNull(applications);
+ }
+
+ // ------------------------------------------------------------------------
+
+ public Collection getApplications() {
+ return applications;
+ }
+
+ @Override
+ public String toString() {
+ return "MultipleApplicationsDetails{" + "applications=" + applications + '}';
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ MultipleApplicationsDetails that = (MultipleApplicationsDetails) o;
+ return Objects.equals(applications, that.applications);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(applications);
+ }
+
+ // ------------------------------------------------------------------------
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/application/ApplicationCancellationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/application/ApplicationCancellationHandler.java
new file mode 100644
index 0000000000000..32db846767460
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/application/ApplicationCancellationHandler.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.rest.handler.application;
+
+import org.apache.flink.api.common.ApplicationID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.FlinkApplicationNotFoundException;
+import org.apache.flink.runtime.messages.FlinkApplicationTerminatedWithoutCancellationException;
+import org.apache.flink.runtime.rest.handler.AbstractRestHandler;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.messages.ApplicationCancellationMessageParameters;
+import org.apache.flink.runtime.rest.messages.ApplicationIDPathParameter;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.EmptyResponseBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+import java.time.Duration;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.TimeoutException;
+
+/** Request handler for the application cancellation request. */
+public class ApplicationCancellationHandler
+ extends AbstractRestHandler<
+ RestfulGateway,
+ EmptyRequestBody,
+ EmptyResponseBody,
+ ApplicationCancellationMessageParameters> {
+
+ public ApplicationCancellationHandler(
+ GatewayRetriever extends RestfulGateway> leaderRetriever,
+ Duration timeout,
+ Map headers,
+ MessageHeaders<
+ EmptyRequestBody,
+ EmptyResponseBody,
+ ApplicationCancellationMessageParameters>
+ messageHeaders) {
+ super(leaderRetriever, timeout, headers, messageHeaders);
+ }
+
+ @Override
+ public CompletableFuture handleRequest(
+ HandlerRequest request, RestfulGateway gateway)
+ throws RestHandlerException {
+ final ApplicationID applicationId =
+ request.getPathParameter(ApplicationIDPathParameter.class);
+
+ final CompletableFuture terminationFuture =
+ gateway.cancelApplication(applicationId, timeout);
+
+ return terminationFuture.handle(
+ (Acknowledge ack, Throwable throwable) -> {
+ if (throwable != null) {
+ Throwable error = ExceptionUtils.stripCompletionException(throwable);
+
+ if (error
+ instanceof FlinkApplicationTerminatedWithoutCancellationException) {
+ throw new CompletionException(
+ new RestHandlerException(
+ String.format(
+ "Application cancellation failed because the application has already reached another terminal state (%s).",
+ ((FlinkApplicationTerminatedWithoutCancellationException)
+ error)
+ .getApplicationStatus()),
+ HttpResponseStatus.CONFLICT));
+ } else if (error instanceof TimeoutException) {
+ throw new CompletionException(
+ new RestHandlerException(
+ "Application cancellation timed out.",
+ HttpResponseStatus.REQUEST_TIMEOUT,
+ error));
+ } else if (error instanceof FlinkApplicationNotFoundException) {
+ throw new CompletionException(
+ new RestHandlerException(
+ "Application could not be found.",
+ HttpResponseStatus.NOT_FOUND,
+ error));
+ } else {
+ throw new CompletionException(
+ new RestHandlerException(
+ "Application cancellation failed: "
+ + error.getMessage(),
+ HttpResponseStatus.INTERNAL_SERVER_ERROR,
+ error));
+ }
+ } else {
+ return EmptyResponseBody.getInstance();
+ }
+ });
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/application/ApplicationDetailsHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/application/ApplicationDetailsHandler.java
new file mode 100644
index 0000000000000..cd0987b47a2f6
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/application/ApplicationDetailsHandler.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.application;
+
+import org.apache.flink.api.common.ApplicationID;
+import org.apache.flink.runtime.messages.webmonitor.ApplicationDetailsInfo;
+import org.apache.flink.runtime.rest.handler.AbstractRestHandler;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.messages.ApplicationIDPathParameter;
+import org.apache.flink.runtime.rest.messages.ApplicationMessageParameters;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+
+import javax.annotation.Nonnull;
+
+import java.time.Duration;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+/** Handler returning the details for the specified application. */
+public class ApplicationDetailsHandler
+ extends AbstractRestHandler<
+ RestfulGateway,
+ EmptyRequestBody,
+ ApplicationDetailsInfo,
+ ApplicationMessageParameters> {
+
+ public ApplicationDetailsHandler(
+ GatewayRetriever extends RestfulGateway> leaderRetriever,
+ Duration timeout,
+ Map responseHeaders,
+ MessageHeaders
+ messageHeaders) {
+ super(leaderRetriever, timeout, responseHeaders, messageHeaders);
+ }
+
+ @Override
+ public CompletableFuture handleRequest(
+ @Nonnull HandlerRequest request, @Nonnull RestfulGateway gateway) {
+ ApplicationID applicationId = request.getPathParameter(ApplicationIDPathParameter.class);
+ return gateway.requestApplication(applicationId, timeout);
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/application/ApplicationsOverviewHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/application/ApplicationsOverviewHandler.java
new file mode 100644
index 0000000000000..47f8ea10ccb94
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/application/ApplicationsOverviewHandler.java
@@ -0,0 +1,60 @@
+/*
+ * 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.handler.application;
+
+import org.apache.flink.runtime.messages.webmonitor.MultipleApplicationsDetails;
+import org.apache.flink.runtime.rest.handler.AbstractRestHandler;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.messages.EmptyMessageParameters;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+
+import javax.annotation.Nonnull;
+
+import java.time.Duration;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+/** Overview handler for applications. */
+public class ApplicationsOverviewHandler
+ extends AbstractRestHandler<
+ RestfulGateway,
+ EmptyRequestBody,
+ MultipleApplicationsDetails,
+ EmptyMessageParameters> {
+
+ public ApplicationsOverviewHandler(
+ GatewayRetriever extends RestfulGateway> leaderRetriever,
+ Duration timeout,
+ Map responseHeaders,
+ MessageHeaders
+ messageHeaders) {
+ super(leaderRetriever, timeout, responseHeaders, messageHeaders);
+ }
+
+ @Override
+ protected CompletableFuture handleRequest(
+ @Nonnull HandlerRequest request, @Nonnull RestfulGateway gateway)
+ throws RestHandlerException {
+ return gateway.requestMultipleApplicationDetails(timeout);
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ApplicationCancellationHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ApplicationCancellationHeaders.java
new file mode 100644
index 0000000000000..b7ee327dd8f73
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ApplicationCancellationHeaders.java
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+import org.apache.flink.runtime.rest.HttpMethodWrapper;
+import org.apache.flink.runtime.rest.handler.application.ApplicationCancellationHandler;
+
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+/** Message headers for the {@link ApplicationCancellationHandler}. */
+public class ApplicationCancellationHeaders
+ implements RuntimeMessageHeaders<
+ EmptyRequestBody, EmptyResponseBody, ApplicationCancellationMessageParameters> {
+
+ public static final String URL = "/applications/:applicationid/cancel";
+
+ private static final ApplicationCancellationHeaders INSTANCE =
+ new ApplicationCancellationHeaders();
+
+ private ApplicationCancellationHeaders() {}
+
+ @Override
+ public Class getRequestClass() {
+ return EmptyRequestBody.class;
+ }
+
+ @Override
+ public Class getResponseClass() {
+ return EmptyResponseBody.class;
+ }
+
+ @Override
+ public HttpResponseStatus getResponseStatusCode() {
+ return HttpResponseStatus.ACCEPTED;
+ }
+
+ @Override
+ public ApplicationCancellationMessageParameters getUnresolvedMessageParameters() {
+ return new ApplicationCancellationMessageParameters();
+ }
+
+ @Override
+ public HttpMethodWrapper getHttpMethod() {
+ return HttpMethodWrapper.POST;
+ }
+
+ @Override
+ public String getTargetRestEndpointURL() {
+ return URL;
+ }
+
+ public static ApplicationCancellationHeaders getInstance() {
+ return INSTANCE;
+ }
+
+ @Override
+ public String getDescription() {
+ return "Terminates an application.";
+ }
+
+ @Override
+ public String operationId() {
+ return "cancelApplication";
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ApplicationCancellationMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ApplicationCancellationMessageParameters.java
new file mode 100644
index 0000000000000..39b019e91c3e1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ApplicationCancellationMessageParameters.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.messages;
+
+import org.apache.flink.api.common.ApplicationID;
+
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * Parameters for application related REST handlers.
+ *
+ * A application related REST handler always requires a {@link ApplicationIDPathParameter}.
+ */
+public class ApplicationCancellationMessageParameters extends MessageParameters {
+
+ private final ApplicationIDPathParameter applicationPathParameter =
+ new ApplicationIDPathParameter();
+
+ @Override
+ public Collection> getPathParameters() {
+ return Collections.singleton(applicationPathParameter);
+ }
+
+ @Override
+ public Collection> getQueryParameters() {
+ return Collections.emptySet();
+ }
+
+ public ApplicationCancellationMessageParameters resolveApplicationId(
+ ApplicationID applicationId) {
+ applicationPathParameter.resolve(applicationId);
+ return this;
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ApplicationIDPathParameter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ApplicationIDPathParameter.java
new file mode 100644
index 0000000000000..e34fb1f174c31
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ApplicationIDPathParameter.java
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.ApplicationID;
+
+/** Path parameter identifying applications. */
+public class ApplicationIDPathParameter extends MessagePathParameter {
+
+ public static final String KEY = "applicationid";
+
+ public ApplicationIDPathParameter() {
+ super(KEY);
+ }
+
+ @Override
+ protected ApplicationID convertFromString(String value) {
+ return ApplicationID.fromHexString(value);
+ }
+
+ @Override
+ protected String convertToString(ApplicationID value) {
+ return value.toString();
+ }
+
+ @Override
+ public String getDescription() {
+ return "32-character hexadecimal string value that identifies an application.";
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ApplicationMessageParameters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ApplicationMessageParameters.java
new file mode 100644
index 0000000000000..c8d4f5624c9ae
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ApplicationMessageParameters.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.messages;
+
+import java.util.Collection;
+import java.util.Collections;
+
+/** Message parameters which require an application path parameter. */
+public class ApplicationMessageParameters extends MessageParameters {
+
+ public final ApplicationIDPathParameter applicationPathParameter =
+ new ApplicationIDPathParameter();
+
+ @Override
+ public Collection> getPathParameters() {
+ return Collections.singleton(applicationPathParameter);
+ }
+
+ @Override
+ public Collection> getQueryParameters() {
+ return Collections.emptySet();
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ApplicationsOverviewHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ApplicationsOverviewHeaders.java
new file mode 100644
index 0000000000000..135a40897b1b5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/ApplicationsOverviewHeaders.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;
+
+import org.apache.flink.runtime.messages.webmonitor.MultipleApplicationsDetails;
+import org.apache.flink.runtime.rest.HttpMethodWrapper;
+import org.apache.flink.runtime.rest.handler.application.ApplicationsOverviewHandler;
+
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+/** Message headers for {@link ApplicationsOverviewHandler}. */
+public final class ApplicationsOverviewHeaders
+ implements RuntimeMessageHeaders<
+ EmptyRequestBody, MultipleApplicationsDetails, EmptyMessageParameters> {
+
+ private static final ApplicationsOverviewHeaders INSTANCE = new ApplicationsOverviewHeaders();
+
+ public static final String URL = "/applications/overview";
+
+ // make this class a singleton
+ private ApplicationsOverviewHeaders() {}
+
+ @Override
+ public Class getRequestClass() {
+ return EmptyRequestBody.class;
+ }
+
+ @Override
+ public HttpMethodWrapper getHttpMethod() {
+ return HttpMethodWrapper.GET;
+ }
+
+ @Override
+ public String getTargetRestEndpointURL() {
+ return URL;
+ }
+
+ @Override
+ public Class getResponseClass() {
+ return MultipleApplicationsDetails.class;
+ }
+
+ @Override
+ public HttpResponseStatus getResponseStatusCode() {
+ return HttpResponseStatus.OK;
+ }
+
+ @Override
+ public EmptyMessageParameters getUnresolvedMessageParameters() {
+ return EmptyMessageParameters.getInstance();
+ }
+
+ public static ApplicationsOverviewHeaders getInstance() {
+ return INSTANCE;
+ }
+
+ @Override
+ public String getDescription() {
+ return "Returns an overview over all applications.";
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/application/ApplicationDetailsHeaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/application/ApplicationDetailsHeaders.java
new file mode 100644
index 0000000000000..3ee9f0ac34c0c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/application/ApplicationDetailsHeaders.java
@@ -0,0 +1,80 @@
+/*
+ * 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.application;
+
+import org.apache.flink.runtime.messages.webmonitor.ApplicationDetailsInfo;
+import org.apache.flink.runtime.rest.HttpMethodWrapper;
+import org.apache.flink.runtime.rest.handler.application.ApplicationDetailsHandler;
+import org.apache.flink.runtime.rest.messages.ApplicationIDPathParameter;
+import org.apache.flink.runtime.rest.messages.ApplicationMessageParameters;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.RuntimeMessageHeaders;
+
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+/** Message headers for the {@link ApplicationDetailsHandler}. */
+public class ApplicationDetailsHeaders
+ implements RuntimeMessageHeaders<
+ EmptyRequestBody, ApplicationDetailsInfo, ApplicationMessageParameters> {
+
+ private static final ApplicationDetailsHeaders INSTANCE = new ApplicationDetailsHeaders();
+
+ public static final String URL = "/applications/:" + ApplicationIDPathParameter.KEY;
+
+ private ApplicationDetailsHeaders() {}
+
+ @Override
+ public Class getRequestClass() {
+ return EmptyRequestBody.class;
+ }
+
+ @Override
+ public Class getResponseClass() {
+ return ApplicationDetailsInfo.class;
+ }
+
+ @Override
+ public HttpResponseStatus getResponseStatusCode() {
+ return HttpResponseStatus.OK;
+ }
+
+ @Override
+ public ApplicationMessageParameters getUnresolvedMessageParameters() {
+ return new ApplicationMessageParameters();
+ }
+
+ @Override
+ public HttpMethodWrapper getHttpMethod() {
+ return HttpMethodWrapper.GET;
+ }
+
+ @Override
+ public String getTargetRestEndpointURL() {
+ return URL;
+ }
+
+ public static ApplicationDetailsHeaders getInstance() {
+ return INSTANCE;
+ }
+
+ @Override
+ public String getDescription() {
+ return "Returns details of an application.";
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/ApplicationIDDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/ApplicationIDDeserializer.java
new file mode 100644
index 0000000000000..f5c4340b0c223
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/ApplicationIDDeserializer.java
@@ -0,0 +1,41 @@
+/*
+ * 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.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+
+import java.io.IOException;
+
+/** Json deserializer for {@link ApplicationID}. */
+public class ApplicationIDDeserializer extends StdDeserializer {
+ private static final long serialVersionUID = -130167416771003558L;
+
+ public ApplicationIDDeserializer() {
+ super(ApplicationID.class);
+ }
+
+ @Override
+ public ApplicationID deserialize(JsonParser p, DeserializationContext ctxt) throws IOException {
+ return ApplicationID.fromHexString(p.getValueAsString());
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/ApplicationIDSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/ApplicationIDSerializer.java
new file mode 100644
index 0000000000000..cac5ed2e7545a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/json/ApplicationIDSerializer.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.rest.messages.json;
+
+import org.apache.flink.api.common.ApplicationID;
+
+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 ApplicationID}. */
+public class ApplicationIDSerializer extends StdSerializer {
+
+ private static final long serialVersionUID = -6598593519161574610L;
+
+ public ApplicationIDSerializer() {
+ super(ApplicationID.class);
+ }
+
+ @Override
+ public void serialize(ApplicationID value, JsonGenerator gen, SerializerProvider provider)
+ throws IOException {
+ gen.writeString(value.toString());
+ }
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/NonLeaderRetrievalRestfulGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/NonLeaderRetrievalRestfulGateway.java
index 90ba4b8ec4086..b815dca274b90 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/NonLeaderRetrievalRestfulGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/NonLeaderRetrievalRestfulGateway.java
@@ -18,13 +18,16 @@
package org.apache.flink.runtime.webmonitor;
+import org.apache.flink.api.common.ApplicationID;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
import org.apache.flink.runtime.clusterframework.types.ResourceID;
import org.apache.flink.runtime.jobmaster.JobResult;
import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.webmonitor.ApplicationDetailsInfo;
import org.apache.flink.runtime.messages.webmonitor.ClusterOverview;
+import org.apache.flink.runtime.messages.webmonitor.MultipleApplicationsDetails;
import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
import org.apache.flink.runtime.rest.messages.ThreadDumpInfo;
import org.apache.flink.runtime.scheduler.ExecutionGraphInfo;
@@ -64,12 +67,24 @@ public CompletableFuture cancelJob(JobID jobId, Duration timeout) {
throw new UnsupportedOperationException(MESSAGE);
}
+ @Override
+ public CompletableFuture cancelApplication(
+ ApplicationID applicationId, Duration timeout) {
+ throw new UnsupportedOperationException(MESSAGE);
+ }
+
@Override
public CompletableFuture requestExecutionGraphInfo(
JobID jobId, Duration timeout) {
throw new UnsupportedOperationException(MESSAGE);
}
+ @Override
+ public CompletableFuture requestApplication(
+ ApplicationID applicationId, Duration timeout) {
+ throw new UnsupportedOperationException(MESSAGE);
+ }
+
@Override
public CompletableFuture requestCheckpointStats(
JobID jobId, Duration timeout) {
@@ -86,6 +101,12 @@ public CompletableFuture requestMultipleJobDetails(Duration
throw new UnsupportedOperationException(MESSAGE);
}
+ @Override
+ public CompletableFuture requestMultipleApplicationDetails(
+ Duration timeout) {
+ throw new UnsupportedOperationException(MESSAGE);
+ }
+
@Override
public CompletableFuture requestClusterOverview(Duration timeout) {
throw new UnsupportedOperationException(MESSAGE);
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java
index 2a926b97f25f6..01e2575aa502c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/RestfulGateway.java
@@ -18,6 +18,7 @@
package org.apache.flink.runtime.webmonitor;
+import org.apache.flink.api.common.ApplicationID;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.api.java.tuple.Tuple2;
@@ -31,8 +32,11 @@
import org.apache.flink.runtime.jobgraph.JobResourceRequirements;
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.messages.FlinkJobNotFoundException;
+import org.apache.flink.runtime.messages.webmonitor.ApplicationDetailsInfo;
import org.apache.flink.runtime.messages.webmonitor.ClusterOverview;
+import org.apache.flink.runtime.messages.webmonitor.MultipleApplicationsDetails;
import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
import org.apache.flink.runtime.metrics.dump.MetricQueryService;
import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
@@ -67,6 +71,16 @@ public interface RestfulGateway extends RpcGateway {
*/
CompletableFuture cancelJob(JobID jobId, @RpcTimeout Duration timeout);
+ /**
+ * Cancel the given application.
+ *
+ * @param applicationId identifying the job to cancel
+ * @param timeout of the operation
+ * @return A future acknowledge if the cancellation succeeded
+ */
+ CompletableFuture cancelApplication(
+ ApplicationID applicationId, @RpcTimeout Duration timeout);
+
/**
* Requests the {@link ArchivedExecutionGraph} for the given jobId. If there is no such graph,
* then the future is completed with a {@link FlinkJobNotFoundException}.
@@ -82,6 +96,19 @@ default CompletableFuture requestJob(
.thenApply(ExecutionGraphInfo::getArchivedExecutionGraph);
}
+ /**
+ * Requests the {@link ApplicationDetailsInfo} for the given applicationId. If there is no such
+ * application, then the future is completed with a {@link FlinkApplicationNotFoundException}.
+ *
+ * @param applicationId identifying the application whose {@link ApplicationDetailsInfo} is
+ * requested
+ * @param timeout for the asynchronous operation
+ * @return Future containing the {@link ApplicationDetailsInfo} for the given applicationId,
+ * otherwise {@link FlinkApplicationNotFoundException}
+ */
+ CompletableFuture requestApplication(
+ ApplicationID applicationId, @RpcTimeout Duration timeout);
+
/**
* Requests the {@link ExecutionGraphInfo} containing additional information besides the {@link
* ArchivedExecutionGraph}. If there is no such graph, then the future is completed with a
@@ -122,6 +149,15 @@ CompletableFuture requestCheckpointStats(
*/
CompletableFuture requestMultipleJobDetails(@RpcTimeout Duration timeout);
+ /**
+ * Requests application details currently being executed on the Flink cluster.
+ *
+ * @param timeout for the asynchronous operation
+ * @return Future containing the application details
+ */
+ CompletableFuture requestMultipleApplicationDetails(
+ @RpcTimeout Duration timeout);
+
/**
* Requests the cluster status overview.
*
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java
index c54631c8faf35..d9f09f131d135 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/webmonitor/WebMonitorEndpoint.java
@@ -33,6 +33,9 @@
import org.apache.flink.runtime.rest.handler.AbstractRestHandler;
import org.apache.flink.runtime.rest.handler.RestHandlerConfiguration;
import org.apache.flink.runtime.rest.handler.RestHandlerSpecification;
+import org.apache.flink.runtime.rest.handler.application.ApplicationCancellationHandler;
+import org.apache.flink.runtime.rest.handler.application.ApplicationDetailsHandler;
+import org.apache.flink.runtime.rest.handler.application.ApplicationsOverviewHandler;
import org.apache.flink.runtime.rest.handler.cluster.ClusterConfigHandler;
import org.apache.flink.runtime.rest.handler.cluster.ClusterOverviewHandler;
import org.apache.flink.runtime.rest.handler.cluster.DashboardConfigHandler;
@@ -107,6 +110,8 @@
import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerStdoutFileHandler;
import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagerThreadDumpHandler;
import org.apache.flink.runtime.rest.handler.taskmanager.TaskManagersHandler;
+import org.apache.flink.runtime.rest.messages.ApplicationCancellationHeaders;
+import org.apache.flink.runtime.rest.messages.ApplicationsOverviewHeaders;
import org.apache.flink.runtime.rest.messages.ClusterConfigurationInfoHeaders;
import org.apache.flink.runtime.rest.messages.ClusterOverviewHeaders;
import org.apache.flink.runtime.rest.messages.DashboardConfigurationHeaders;
@@ -130,6 +135,7 @@
import org.apache.flink.runtime.rest.messages.TerminationModeQueryParameter;
import org.apache.flink.runtime.rest.messages.YarnCancelJobTerminationHeaders;
import org.apache.flink.runtime.rest.messages.YarnStopJobTerminationHeaders;
+import org.apache.flink.runtime.rest.messages.application.ApplicationDetailsHeaders;
import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointConfigHeaders;
import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointStatisticDetailsHeaders;
import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointingStatisticsHeaders;
@@ -341,6 +347,13 @@ protected List> initiali
responseHeaders,
JobsOverviewHeaders.getInstance());
+ ApplicationsOverviewHandler applicationsOverviewHandler =
+ new ApplicationsOverviewHandler(
+ leaderRetriever,
+ timeout,
+ responseHeaders,
+ ApplicationsOverviewHeaders.getInstance());
+
ClusterConfigHandler clusterConfigurationHandler =
new ClusterConfigHandler(
leaderRetriever,
@@ -481,6 +494,13 @@ protected List> initiali
executor,
metricFetcher);
+ ApplicationDetailsHandler applicationDetailsHandler =
+ new ApplicationDetailsHandler(
+ leaderRetriever,
+ timeout,
+ responseHeaders,
+ ApplicationDetailsHeaders.getInstance());
+
JobAccumulatorsHandler jobAccumulatorsHandler =
new JobAccumulatorsHandler(
leaderRetriever,
@@ -629,6 +649,13 @@ protected List> initiali
JobVertexBackPressureHeaders.getInstance(),
metricFetcher);
+ final ApplicationCancellationHandler applicationCancellationHandler =
+ new ApplicationCancellationHandler(
+ leaderRetriever,
+ timeout,
+ responseHeaders,
+ ApplicationCancellationHeaders.getInstance());
+
final JobCancellationHandler jobCancelTerminationHandler =
new JobCancellationHandler(
leaderRetriever,
@@ -753,6 +780,10 @@ protected List> initiali
handlers.add(Tuple2.of(jobIdsHandler.getMessageHeaders(), jobIdsHandler));
handlers.add(Tuple2.of(jobStatusHandler.getMessageHeaders(), jobStatusHandler));
handlers.add(Tuple2.of(jobsOverviewHandler.getMessageHeaders(), jobsOverviewHandler));
+ handlers.add(
+ Tuple2.of(
+ applicationsOverviewHandler.getMessageHeaders(),
+ applicationsOverviewHandler));
handlers.add(Tuple2.of(jobConfigHandler.getMessageHeaders(), jobConfigHandler));
handlers.add(
Tuple2.of(checkpointConfigHandler.getMessageHeaders(), checkpointConfigHandler));
@@ -779,6 +810,9 @@ protected List> initiali
subtasksAllAccumulatorsHandler.getMessageHeaders(),
subtasksAllAccumulatorsHandler));
handlers.add(Tuple2.of(jobDetailsHandler.getMessageHeaders(), jobDetailsHandler));
+ handlers.add(
+ Tuple2.of(
+ applicationDetailsHandler.getMessageHeaders(), applicationDetailsHandler));
handlers.add(Tuple2.of(jobAccumulatorsHandler.getMessageHeaders(), jobAccumulatorsHandler));
handlers.add(Tuple2.of(taskManagersHandler.getMessageHeaders(), taskManagersHandler));
handlers.add(
@@ -873,6 +907,10 @@ protected List> initiali
jobVertexFlameGraphHandler.getMessageHeaders(),
jobVertexFlameGraphHandler));
+ handlers.add(
+ Tuple2.of(
+ applicationCancellationHandler.getMessageHeaders(),
+ applicationCancellationHandler));
handlers.add(
Tuple2.of(
jobCancelTerminationHandler.getMessageHeaders(),
diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/AdaptiveGraphManager.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/AdaptiveGraphManager.java
index 7e91e123ca11d..05d430ff8debf 100644
--- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/AdaptiveGraphManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/AdaptiveGraphManager.java
@@ -188,7 +188,11 @@ public AdaptiveGraphManager(
userClassloader,
this);
- this.jobGraph = createAndInitializeJobGraph(streamGraph, streamGraph.getJobID());
+ this.jobGraph =
+ createAndInitializeJobGraph(
+ streamGraph,
+ streamGraph.getJobID(),
+ streamGraph.getApplicationId().orElse(null));
this.defaultSlotSharingGroup = new SlotSharingGroup();
diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/ExecutionPlan.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/ExecutionPlan.java
index 8b4216fae18a6..48e34f049a5fb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/ExecutionPlan.java
+++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/ExecutionPlan.java
@@ -19,6 +19,7 @@
package org.apache.flink.streaming.api.graph;
import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.ApplicationID;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.cache.DistributedCache;
@@ -35,6 +36,7 @@
import java.net.URL;
import java.util.List;
import java.util.Map;
+import java.util.Optional;
/**
* An interface representing a general execution plan, which can be implemented by different types
@@ -199,4 +201,11 @@ default boolean isCheckpointingEnabled() {
* @return The serialized execution configuration object
*/
SerializedValue getSerializedExecutionConfig();
+
+ /**
+ * Gets the unique identifier of the application this job belongs to.
+ *
+ * @return the application id, or empty if not associated with an application
+ */
+ Optional getApplicationId();
}
diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
index a64d534ba9313..349bf0a16cf31 100644
--- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
@@ -19,6 +19,7 @@
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.ExecutionConfig;
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.JobID;
@@ -134,6 +135,9 @@ public class StreamGraph implements Pipeline, ExecutionPlan {
private JobID jobId;
+ /** ID of the application this job belongs to. */
+ @Nullable private ApplicationID applicationId;
+
private final Configuration jobConfiguration;
private transient ExecutionConfig executionConfig;
private final CheckpointConfig checkpointConfig;
@@ -1160,16 +1164,28 @@ public StreamNode getTargetVertex(StreamEdge edge) {
/** Gets the assembled {@link JobGraph} with a random {@link JobID}. */
@VisibleForTesting
public JobGraph getJobGraph() {
- return getJobGraph(Thread.currentThread().getContextClassLoader(), jobId);
+ return getJobGraph(Thread.currentThread().getContextClassLoader(), jobId, applicationId);
}
public JobGraph getJobGraph(ClassLoader userClassLoader) {
- return getJobGraph(userClassLoader, jobId);
+ return getJobGraph(userClassLoader, jobId, applicationId);
}
/** Gets the assembled {@link JobGraph} with a specified {@link JobID}. */
- public JobGraph getJobGraph(ClassLoader userClassLoader, @Nullable JobID jobID) {
- return StreamingJobGraphGenerator.createJobGraph(userClassLoader, this, jobID);
+ public JobGraph getJobGraph(ClassLoader userClassLoader, @Nullable JobID jobId) {
+ return getJobGraph(userClassLoader, jobId, applicationId);
+ }
+
+ /**
+ * Gets the assembled {@link JobGraph} with a specified {@link JobID} and a specified {@link
+ * ApplicationID}.
+ */
+ public JobGraph getJobGraph(
+ ClassLoader userClassLoader,
+ @Nullable JobID jobId,
+ @Nullable ApplicationID applicationId) {
+ return StreamingJobGraphGenerator.createJobGraph(
+ userClassLoader, this, jobId, applicationId);
}
public String getStreamingPlanAsJSON() {
@@ -1266,6 +1282,15 @@ public JobID getJobID() {
return jobId;
}
+ public void setApplicationId(ApplicationID applicationId) {
+ this.applicationId = checkNotNull(applicationId);
+ }
+
+ @Override
+ public Optional getApplicationId() {
+ return Optional.ofNullable(applicationId);
+ }
+
/**
* Sets the classpath required to run the job on a task manager.
*
diff --git a/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
index ae948dc3852f8..14ee3fdffc1c2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
@@ -19,6 +19,7 @@
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.ExecutionConfig;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.RuntimeExecutionMode;
@@ -134,12 +135,16 @@ public static JobGraph createJobGraph(StreamGraph streamGraph) {
Thread.currentThread().getContextClassLoader(),
streamGraph,
streamGraph.getJobID(),
+ streamGraph.getApplicationId().orElse(null),
Runnable::run)
.createJobGraph();
}
public static JobGraph createJobGraph(
- ClassLoader userClassLoader, StreamGraph streamGraph, @Nullable JobID jobID) {
+ ClassLoader userClassLoader,
+ StreamGraph streamGraph,
+ @Nullable JobID jobId,
+ @Nullable ApplicationID applicationId) {
// TODO Currently, we construct a new thread pool for the compilation of each job. In the
// future, we may refactor the job submission framework and make it reusable across jobs.
final ExecutorService serializationExecutor =
@@ -152,7 +157,11 @@ public static JobGraph createJobGraph(
new ExecutorThreadFactory("flink-operator-serialization-io"));
try {
return new StreamingJobGraphGenerator(
- userClassLoader, streamGraph, jobID, serializationExecutor)
+ userClassLoader,
+ streamGraph,
+ jobId,
+ applicationId,
+ serializationExecutor)
.createJobGraph();
} finally {
serializationExecutor.shutdown();
@@ -177,14 +186,15 @@ public static JobGraph createJobGraph(
private StreamingJobGraphGenerator(
ClassLoader userClassloader,
StreamGraph streamGraph,
- @Nullable JobID jobID,
+ @Nullable JobID jobId,
+ @Nullable ApplicationID applicationId,
Executor serializationExecutor) {
this.userClassloader = userClassloader;
this.streamGraph = streamGraph;
this.defaultStreamGraphHasher = new StreamGraphHasherV2();
this.legacyStreamGraphHashers = Arrays.asList(new StreamGraphUserHashHasher());
this.serializationExecutor = Preconditions.checkNotNull(serializationExecutor);
- jobGraph = createAndInitializeJobGraph(streamGraph, jobID);
+ jobGraph = createAndInitializeJobGraph(streamGraph, jobId, applicationId);
// Generate deterministic hashes for the nodes in order to identify them across
// submission iff they didn't change.
@@ -886,8 +896,8 @@ private void setVertexParallelismsForDynamicGraphIfNecessary() {
}
public static JobGraph createAndInitializeJobGraph(
- StreamGraph streamGraph, @Nullable JobID jobId) {
- JobGraph jobGraph = new JobGraph(jobId, streamGraph.getJobName());
+ StreamGraph streamGraph, @Nullable JobID jobId, @Nullable ApplicationID applicationId) {
+ JobGraph jobGraph = new JobGraph(jobId, applicationId, streamGraph.getJobName());
jobGraph.setJobType(streamGraph.getJobType());
jobGraph.setDynamic(streamGraph.isDynamic());
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 ab25cf02dd07c..4e53397a46a07 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
@@ -18,6 +18,7 @@
package org.apache.flink.runtime.dispatcher;
+import org.apache.flink.api.common.ApplicationID;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.api.common.operators.ResourceSpec;
@@ -28,6 +29,7 @@
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.blob.BlobServer;
import org.apache.flink.runtime.blob.PermanentBlobKey;
import org.apache.flink.runtime.checkpoint.CheckpointStatsSnapshot;
@@ -101,6 +103,7 @@
import org.apache.flink.util.InstantiationUtil;
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.concurrent.FutureUtils;
+import org.apache.flink.util.concurrent.ScheduledExecutor;
import org.apache.flink.shaded.guava33.com.google.common.collect.ImmutableMap;
import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
@@ -137,6 +140,7 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
@@ -935,6 +939,43 @@ public void testJobCleanupWithoutRecoveredJobGraph() throws Exception {
.isZero();
}
+ @Test
+ public void testApplicationBootstrap() throws Exception {
+ final OneShotLatch bootstrapLatch = new OneShotLatch();
+ final ApplicationID applicationId = new ApplicationID();
+ final AbstractApplication application =
+ new TestingApplication(
+ applicationId,
+ (ignored -> {
+ bootstrapLatch.trigger();
+ return CompletableFuture.completedFuture(Acknowledge.get());
+ }));
+ dispatcher =
+ createTestingDispatcherBuilder()
+ .setDispatcherBootstrapFactory(
+ (ignoredDispatcherGateway,
+ ignoredScheduledExecutor,
+ ignoredFatalErrorHandler) ->
+ new ApplicationBootstrap(application))
+ .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);
+
+ jobGraph.setApplicationId(applicationId);
+ final DispatcherGateway dispatcherGateway =
+ dispatcher.getSelfGateway(DispatcherGateway.class);
+ dispatcherGateway.submitJob(jobGraph, TIMEOUT).get();
+
+ assertThat(application.getJobs().size()).isEqualTo(1);
+ assertThat(application.getJobs()).contains(jobGraph.getJobID());
+ }
+
@Test
public void testPersistedJobGraphWhenDispatcherIsShutDown() throws Exception {
final TestingExecutionPlanStore submittedExecutionPlanStore =
@@ -2104,4 +2145,57 @@ public JobManagerRunner createJobManagerRunner(
return runner;
}
}
+
+ private static class TestingApplication extends AbstractApplication {
+ private final Function> executeFunction;
+
+ public TestingApplication(
+ ApplicationID applicationId,
+ Function> executeFunction) {
+ super(applicationId);
+ this.executeFunction = executeFunction;
+ }
+
+ @Override
+ public CompletableFuture execute(
+ DispatcherGateway dispatcherGateway,
+ ScheduledExecutor scheduledExecutor,
+ Executor mainThreadExecutor,
+ FatalErrorHandler errorHandler) {
+
+ ExecuteParams params =
+ new ExecuteParams(
+ dispatcherGateway, scheduledExecutor, mainThreadExecutor, errorHandler);
+ return executeFunction.apply(params);
+ }
+
+ @Override
+ public void cancel() {}
+
+ @Override
+ public void dispose() {}
+
+ @Override
+ public String getName() {
+ return "TestingApplication";
+ }
+
+ public static class ExecuteParams {
+ public final DispatcherGateway dispatcherGateway;
+ public final ScheduledExecutor scheduledExecutor;
+ public final Executor mainThreadExecutor;
+ public final FatalErrorHandler errorHandler;
+
+ public ExecuteParams(
+ DispatcherGateway dispatcherGateway,
+ ScheduledExecutor scheduledExecutor,
+ Executor mainThreadExecutor,
+ FatalErrorHandler errorHandler) {
+ this.dispatcherGateway = dispatcherGateway;
+ this.scheduledExecutor = scheduledExecutor;
+ this.mainThreadExecutor = mainThreadExecutor;
+ this.errorHandler = errorHandler;
+ }
+ }
+ }
}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/GenericMessageTester.java b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/GenericMessageTester.java
index 2b80f6456b7a5..f7411d408eec6 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/GenericMessageTester.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/GenericMessageTester.java
@@ -18,6 +18,8 @@
package org.apache.flink.runtime.messages;
+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.core.testutils.CommonTestUtils;
@@ -128,10 +130,18 @@ public static JobID randomJobId(Random rnd) {
return new JobID(rnd.nextLong(), rnd.nextLong());
}
+ public static ApplicationID randomApplicationId(Random rnd) {
+ return new ApplicationID(rnd.nextLong(), rnd.nextLong());
+ }
+
public static JobStatus randomJobStatus(Random rnd) {
return JobStatus.values()[rnd.nextInt(JobStatus.values().length)];
}
+ public static ApplicationState randomApplicationState(Random rnd) {
+ return ApplicationState.values()[rnd.nextInt(ApplicationState.values().length)];
+ }
+
// ------------------------------------------------------------------------
// Map of Instantiators
// ------------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/WebMonitorMessagesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/WebMonitorMessagesTest.java
index 81a6d56a23dcb..b0858b48c62ec 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/messages/WebMonitorMessagesTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/messages/WebMonitorMessagesTest.java
@@ -18,23 +18,31 @@
package org.apache.flink.runtime.messages;
+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.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.messages.webmonitor.ApplicationDetails;
+import org.apache.flink.runtime.messages.webmonitor.ApplicationDetailsInfo;
import org.apache.flink.runtime.messages.webmonitor.ClusterOverview;
import org.apache.flink.runtime.messages.webmonitor.JobDetails;
import org.apache.flink.runtime.messages.webmonitor.JobIdsWithStatusOverview;
import org.apache.flink.runtime.messages.webmonitor.JobsOverview;
+import org.apache.flink.runtime.messages.webmonitor.MultipleApplicationsDetails;
import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
import org.apache.flink.runtime.messages.webmonitor.RequestJobDetails;
import org.apache.flink.runtime.messages.webmonitor.RequestJobsOverview;
import org.apache.flink.runtime.messages.webmonitor.RequestJobsWithIDsOverview;
import org.apache.flink.runtime.messages.webmonitor.RequestStatusOverview;
+import org.apache.flink.util.CollectionUtil;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
import java.util.Random;
class WebMonitorMessagesTest {
@@ -157,4 +165,106 @@ private Collection randomJobDetails(Random rnd) {
}
return Arrays.asList(details);
}
+
+ @Test
+ void testApplicationDetailsMessage() throws Exception {
+ final Random rnd = new Random();
+
+ Map jobInfo = new HashMap<>();
+
+ for (int i = 0; i < JobStatus.values().length; i++) {
+ int count = rnd.nextInt(55);
+ jobInfo.put(JobStatus.values()[i].name(), count);
+ }
+
+ long time = rnd.nextLong();
+ long endTime = rnd.nextBoolean() ? -1L : time + rnd.nextInt();
+
+ String name = GenericMessageTester.randomString(rnd);
+ ApplicationID id = GenericMessageTester.randomApplicationId(rnd);
+ ApplicationState status = GenericMessageTester.randomApplicationState(rnd);
+
+ ApplicationDetails msg1 =
+ new ApplicationDetails(
+ id, name, time, endTime, endTime - time, status.toString(), jobInfo);
+ ApplicationDetails msg2 =
+ new ApplicationDetails(
+ id, name, time, endTime, endTime - time, status.toString(), jobInfo);
+
+ GenericMessageTester.testMessageInstances(msg1, msg2);
+ }
+
+ @Test
+ void testApplicationDetailsInfoMessage() throws Exception {
+ final Random rnd = new Random();
+
+ Map timestamps =
+ CollectionUtil.newHashMapWithExpectedSize(ApplicationState.values().length);
+
+ for (ApplicationState status : ApplicationState.values()) {
+ timestamps.put(status.toString(), rnd.nextLong());
+ }
+
+ long time = rnd.nextLong();
+ long endTime = rnd.nextBoolean() ? -1L : time + rnd.nextInt();
+
+ String name = GenericMessageTester.randomString(rnd);
+ ApplicationID id = GenericMessageTester.randomApplicationId(rnd);
+ ApplicationState status = GenericMessageTester.randomApplicationState(rnd);
+ Collection jobs = randomJobDetails(rnd);
+
+ ApplicationDetailsInfo msg1 =
+ new ApplicationDetailsInfo(
+ id,
+ name,
+ status.toString(),
+ time,
+ endTime,
+ endTime - time,
+ timestamps,
+ jobs);
+ ApplicationDetailsInfo msg2 =
+ new ApplicationDetailsInfo(
+ id,
+ name,
+ status.toString(),
+ time,
+ endTime,
+ endTime - time,
+ timestamps,
+ jobs);
+
+ GenericMessageTester.testMessageInstances(msg1, msg2);
+ }
+
+ @Test
+ void testMultipleApplicationDetails() throws Exception {
+ final Random rnd = new Random();
+ GenericMessageTester.testMessageInstance(
+ new MultipleApplicationsDetails(randomApplicationDetails(rnd)));
+ }
+
+ private Collection randomApplicationDetails(Random rnd) {
+ final ApplicationDetails[] details = new ApplicationDetails[rnd.nextInt(10)];
+ for (int k = 0; k < details.length; k++) {
+ Map jobInfo = new HashMap<>();
+
+ for (int i = 0; i < JobStatus.values().length; i++) {
+ int count = rnd.nextInt(55);
+ jobInfo.put(JobStatus.values()[i].name(), count);
+ }
+
+ long time = rnd.nextLong();
+ long endTime = rnd.nextBoolean() ? -1L : time + rnd.nextInt();
+
+ String name = GenericMessageTester.randomString(rnd);
+ ApplicationID id = GenericMessageTester.randomApplicationId(rnd);
+ ApplicationState status = GenericMessageTester.randomApplicationState(rnd);
+
+ details[k] =
+ new ApplicationDetails(
+ id, name, time, endTime, endTime - time, status.toString(), jobInfo);
+ }
+ return Arrays.asList(details);
+ }
}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/application/ApplicationCancellationHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/application/ApplicationCancellationHandlerTest.java
new file mode 100644
index 0000000000000..8431cf9e97a1f
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/application/ApplicationCancellationHandlerTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.handler.application;
+
+import org.apache.flink.api.common.ApplicationID;
+import org.apache.flink.api.common.ApplicationState;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.FlinkApplicationNotFoundException;
+import org.apache.flink.runtime.messages.FlinkApplicationTerminatedWithoutCancellationException;
+import org.apache.flink.runtime.rest.RestMatchers;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.messages.ApplicationCancellationHeaders;
+import org.apache.flink.runtime.rest.messages.ApplicationCancellationMessageParameters;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.EmptyResponseBody;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.TestingRestfulGateway;
+import org.apache.flink.util.concurrent.FutureUtils;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+import org.junit.jupiter.api.Test;
+
+import java.time.Duration;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Function;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.HamcrestCondition.matching;
+
+/** Tests for the {@link ApplicationCancellationHandler}. */
+class ApplicationCancellationHandlerTest {
+ @Test
+ void testSuccessfulCancellation() throws Exception {
+ testResponse(
+ applicationId -> CompletableFuture.completedFuture(Acknowledge.get()),
+ CompletableFuture::get);
+ }
+
+ @Test
+ void testErrorCodeForNonCanceledTerminalApplication() throws Exception {
+ testResponseCodeOnFailedDispatcherCancellationResponse(
+ applicationId ->
+ FutureUtils.completedExceptionally(
+ new FlinkApplicationTerminatedWithoutCancellationException(
+ applicationId, ApplicationState.FINISHED)),
+ HttpResponseStatus.CONFLICT);
+ }
+
+ @Test
+ void testErrorCodeForTimeout() throws Exception {
+ testResponseCodeOnFailedDispatcherCancellationResponse(
+ applicationId -> FutureUtils.completedExceptionally(new TimeoutException()),
+ HttpResponseStatus.REQUEST_TIMEOUT);
+ }
+
+ @Test
+ void testErrorCodeForUnknownApplication() throws Exception {
+ testResponseCodeOnFailedDispatcherCancellationResponse(
+ applicationId ->
+ FutureUtils.completedExceptionally(
+ new FlinkApplicationNotFoundException(applicationId)),
+ HttpResponseStatus.NOT_FOUND);
+ }
+
+ @Test
+ void testErrorCodeForUnknownError() throws Exception {
+ testResponseCodeOnFailedDispatcherCancellationResponse(
+ applicationId -> FutureUtils.completedExceptionally(new RuntimeException()),
+ HttpResponseStatus.INTERNAL_SERVER_ERROR);
+ }
+
+ private static void testResponseCodeOnFailedDispatcherCancellationResponse(
+ Function> cancelApplicationFunction,
+ HttpResponseStatus expectedErrorCode)
+ throws Exception {
+
+ testResponse(
+ cancelApplicationFunction,
+ cancellationFuture ->
+ assertThat(cancellationFuture)
+ .satisfies(
+ matching(
+ RestMatchers.respondsWithError(
+ expectedErrorCode))));
+ }
+
+ private static void testResponse(
+ Function> cancelApplicationFunction,
+ ThrowingConsumer, Exception> assertion)
+ throws Exception {
+ final RestfulGateway gateway = createGateway(cancelApplicationFunction);
+
+ final ApplicationCancellationHandler applicationCancellationHandler =
+ createHandler(gateway);
+
+ final ApplicationCancellationMessageParameters messageParameters =
+ applicationCancellationHandler
+ .getMessageHeaders()
+ .getUnresolvedMessageParameters()
+ .resolveApplicationId(new ApplicationID());
+
+ final CompletableFuture cancellationFuture =
+ applicationCancellationHandler.handleRequest(
+ HandlerRequest.create(EmptyRequestBody.getInstance(), messageParameters),
+ gateway);
+
+ assertion.accept(cancellationFuture);
+ }
+
+ private static RestfulGateway createGateway(
+ Function> cancelApplicationFunction) {
+ return new TestingRestfulGateway.Builder()
+ .setCancelApplicationFunction(cancelApplicationFunction)
+ .build();
+ }
+
+ private static ApplicationCancellationHandler createHandler(RestfulGateway gateway) {
+ return new ApplicationCancellationHandler(
+ () -> CompletableFuture.completedFuture(gateway),
+ Duration.ofHours(1),
+ Collections.emptyMap(),
+ ApplicationCancellationHeaders.getInstance());
+ }
+}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingDispatcherGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingDispatcherGateway.java
index 7b9cfa773be63..625d849a395a1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingDispatcherGateway.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingDispatcherGateway.java
@@ -18,6 +18,7 @@
package org.apache.flink.runtime.webmonitor;
+import org.apache.flink.api.common.ApplicationID;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.api.java.tuple.Tuple2;
@@ -32,7 +33,9 @@
import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
import org.apache.flink.runtime.jobmaster.JobResult;
import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.webmonitor.ApplicationDetailsInfo;
import org.apache.flink.runtime.messages.webmonitor.ClusterOverview;
+import org.apache.flink.runtime.messages.webmonitor.MultipleApplicationsDetails;
import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
@@ -129,7 +132,10 @@ public TestingDispatcherGateway(
String address,
String hostname,
Function> cancelJobFunction,
+ Function> cancelApplicationFunction,
Function> requestJobFunction,
+ Function>
+ requestApplicationFunction,
Function>
requestExecutionGraphInfoFunction,
Function>
@@ -137,6 +143,8 @@ public TestingDispatcherGateway(
Function> requestJobResultFunction,
Function> requestJobStatusFunction,
Supplier> requestMultipleJobDetailsSupplier,
+ Supplier>
+ requestMultipleApplicationDetailsSupplier,
Supplier> requestClusterOverviewSupplier,
Supplier>>
requestMetricQueryServiceAddressesSupplier,
@@ -187,12 +195,15 @@ public TestingDispatcherGateway(
address,
hostname,
cancelJobFunction,
+ cancelApplicationFunction,
requestJobFunction,
+ requestApplicationFunction,
requestExecutionGraphInfoFunction,
requestCheckpointStatsSnapshotFunction,
requestJobResultFunction,
requestJobStatusFunction,
requestMultipleJobDetailsSupplier,
+ requestMultipleApplicationDetailsSupplier,
requestClusterOverviewSupplier,
requestMetricQueryServiceAddressesSupplier,
requestTaskManagerMetricQueryServiceGatewaysSupplier,
@@ -399,12 +410,15 @@ public TestingDispatcherGateway build() {
address,
hostname,
cancelJobFunction,
+ cancelApplicationFunction,
requestJobFunction,
+ requestApplicationFunction,
requestExecutionGraphInfoFunction,
requestCheckpointStatsSnapshotFunction,
requestJobResultFunction,
requestJobStatusFunction,
requestMultipleJobDetailsSupplier,
+ requestMultipleApplicationDetailsSupplier,
requestClusterOverviewSupplier,
requestMetricQueryServiceGatewaysSupplier,
requestTaskManagerMetricQueryServiceGatewaysSupplier,
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java
index 2e0029cf8cb1e..d62cdb6f899f3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/webmonitor/TestingRestfulGateway.java
@@ -18,6 +18,7 @@
package org.apache.flink.runtime.webmonitor;
+import org.apache.flink.api.common.ApplicationID;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.api.java.tuple.Tuple2;
@@ -29,8 +30,10 @@
import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph;
import org.apache.flink.runtime.jobmaster.JobResult;
import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.webmonitor.ApplicationDetailsInfo;
import org.apache.flink.runtime.messages.webmonitor.ClusterOverview;
import org.apache.flink.runtime.messages.webmonitor.JobsOverview;
+import org.apache.flink.runtime.messages.webmonitor.MultipleApplicationsDetails;
import org.apache.flink.runtime.messages.webmonitor.MultipleJobsDetails;
import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
@@ -55,12 +58,19 @@ public class TestingRestfulGateway implements RestfulGateway {
static final Function> DEFAULT_CANCEL_JOB_FUNCTION =
jobId -> CompletableFuture.completedFuture(Acknowledge.get());
+ static final Function>
+ DEFAULT_CANCEL_APPLICATION_FUNCTION =
+ applicationId -> CompletableFuture.completedFuture(Acknowledge.get());
static final Function> DEFAULT_REQUEST_JOB_RESULT_FUNCTION =
jobId -> FutureUtils.completedExceptionally(new UnsupportedOperationException());
static final Function>
DEFAULT_REQUEST_JOB_FUNCTION =
jobId ->
FutureUtils.completedExceptionally(new UnsupportedOperationException());
+ static final Function>
+ DEFAULT_REQUEST_APPLICATION_FUNCTION =
+ applicationId ->
+ FutureUtils.completedExceptionally(new UnsupportedOperationException());
static final Function>
DEFAULT_REQUEST_EXECUTION_GRAPH_INFO =
jobId ->
@@ -76,6 +86,11 @@ public class TestingRestfulGateway implements RestfulGateway {
() ->
CompletableFuture.completedFuture(
new MultipleJobsDetails(Collections.emptyList()));
+ static final Supplier>
+ DEFAULT_REQUEST_MULTIPLE_APPLICATION_DETAILS_SUPPLIER =
+ () ->
+ CompletableFuture.completedFuture(
+ new MultipleApplicationsDetails(Collections.emptyList()));
static final Supplier>
DEFAULT_REQUEST_CLUSTER_OVERVIEW_SUPPLIER =
() ->
@@ -145,10 +160,15 @@ public class TestingRestfulGateway implements RestfulGateway {
protected Function> cancelJobFunction;
+ protected Function> cancelApplicationFunction;
+
protected Supplier> clusterShutdownSupplier;
protected Function> requestJobFunction;
+ protected Function>
+ requestApplicationFunction;
+
protected Function>
requestExecutionGraphInfoFunction;
@@ -161,6 +181,9 @@ public class TestingRestfulGateway implements RestfulGateway {
protected Supplier> requestMultipleJobDetailsSupplier;
+ protected Supplier>
+ requestMultipleApplicationDetailsSupplier;
+
protected Supplier> requestClusterOverviewSupplier;
protected Supplier>>
@@ -207,12 +230,15 @@ public TestingRestfulGateway() {
LOCALHOST,
LOCALHOST,
DEFAULT_CANCEL_JOB_FUNCTION,
+ DEFAULT_CANCEL_APPLICATION_FUNCTION,
DEFAULT_REQUEST_JOB_FUNCTION,
+ DEFAULT_REQUEST_APPLICATION_FUNCTION,
DEFAULT_REQUEST_EXECUTION_GRAPH_INFO,
DEFAULT_REQUEST_CHECKPOINT_STATS_SNAPSHOT,
DEFAULT_REQUEST_JOB_RESULT_FUNCTION,
DEFAULT_REQUEST_JOB_STATUS_FUNCTION,
DEFAULT_REQUEST_MULTIPLE_JOB_DETAILS_SUPPLIER,
+ DEFAULT_REQUEST_MULTIPLE_APPLICATION_DETAILS_SUPPLIER,
DEFAULT_REQUEST_CLUSTER_OVERVIEW_SUPPLIER,
DEFAULT_REQUEST_METRIC_QUERY_SERVICE_PATHS_SUPPLIER,
DEFAULT_REQUEST_TASK_MANAGER_METRIC_QUERY_SERVICE_PATHS_SUPPLIER,
@@ -230,7 +256,10 @@ public TestingRestfulGateway(
String address,
String hostname,
Function> cancelJobFunction,
+ Function> cancelApplicationFunction,
Function> requestJobFunction,
+ Function>
+ requestApplicationFunction,
Function>
requestExecutionGraphInfoFunction,
Function>
@@ -238,6 +267,8 @@ public TestingRestfulGateway(
Function> requestJobResultFunction,
Function> requestJobStatusFunction,
Supplier> requestMultipleJobDetailsSupplier,
+ Supplier>
+ requestMultipleApplicationDetailsSupplier,
Supplier> requestClusterOverviewSupplier,
Supplier>>
requestMetricQueryServiceAddressesSupplier,
@@ -272,12 +303,15 @@ public TestingRestfulGateway(
this.address = address;
this.hostname = hostname;
this.cancelJobFunction = cancelJobFunction;
+ this.cancelApplicationFunction = cancelApplicationFunction;
this.requestJobFunction = requestJobFunction;
+ this.requestApplicationFunction = requestApplicationFunction;
this.requestExecutionGraphInfoFunction = requestExecutionGraphInfoFunction;
this.requestCheckpointStatsSnapshotFunction = requestCheckpointStatsSnapshotFunction;
this.requestJobResultFunction = requestJobResultFunction;
this.requestJobStatusFunction = requestJobStatusFunction;
this.requestMultipleJobDetailsSupplier = requestMultipleJobDetailsSupplier;
+ this.requestMultipleApplicationDetailsSupplier = requestMultipleApplicationDetailsSupplier;
this.requestClusterOverviewSupplier = requestClusterOverviewSupplier;
this.requestMetricQueryServiceAddressesSupplier =
requestMetricQueryServiceAddressesSupplier;
@@ -299,6 +333,12 @@ public CompletableFuture cancelJob(JobID jobId, Duration timeout) {
return cancelJobFunction.apply(jobId);
}
+ @Override
+ public CompletableFuture cancelApplication(
+ ApplicationID applicationId, Duration timeout) {
+ return cancelApplicationFunction.apply(applicationId);
+ }
+
@Override
public CompletableFuture shutDownCluster() {
return clusterShutdownSupplier.get();
@@ -309,6 +349,12 @@ public CompletableFuture requestJob(JobID jobId, Duratio
return requestJobFunction.apply(jobId);
}
+ @Override
+ public CompletableFuture requestApplication(
+ ApplicationID applicationId, Duration timeout) {
+ return requestApplicationFunction.apply(applicationId);
+ }
+
@Override
public CompletableFuture requestExecutionGraphInfo(
JobID jobId, Duration timeout) {
@@ -336,6 +382,12 @@ public CompletableFuture requestMultipleJobDetails(Duration
return requestMultipleJobDetailsSupplier.get();
}
+ @Override
+ public CompletableFuture requestMultipleApplicationDetails(
+ Duration timeout) {
+ return requestMultipleApplicationDetailsSupplier.get();
+ }
+
@Override
public CompletableFuture requestClusterOverview(Duration timeout) {
return requestClusterOverviewSupplier.get();
@@ -427,7 +479,10 @@ protected abstract static class AbstractBuilder {
protected String address = LOCALHOST;
protected String hostname = LOCALHOST;
protected Function> cancelJobFunction;
+ protected Function> cancelApplicationFunction;
protected Function> requestJobFunction;
+ protected Function>
+ requestApplicationFunction;
protected Function>
requestExecutionGraphInfoFunction;
protected Function>
@@ -436,6 +491,8 @@ protected abstract static class AbstractBuilder