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 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 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 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 { protected Function> requestJobStatusFunction; protected Supplier> requestMultipleJobDetailsSupplier; + protected Supplier> + requestMultipleApplicationDetailsSupplier; protected Supplier> requestClusterOverviewSupplier; protected Supplier> requestOverviewForAllJobsSupplier; protected Supplier>> @@ -472,12 +529,16 @@ protected abstract static class AbstractBuilder { protected AbstractBuilder() { cancelJobFunction = DEFAULT_CANCEL_JOB_FUNCTION; + cancelApplicationFunction = DEFAULT_CANCEL_APPLICATION_FUNCTION; requestJobFunction = DEFAULT_REQUEST_JOB_FUNCTION; + requestApplicationFunction = DEFAULT_REQUEST_APPLICATION_FUNCTION; requestExecutionGraphInfoFunction = DEFAULT_REQUEST_EXECUTION_GRAPH_INFO; requestCheckpointStatsSnapshotFunction = DEFAULT_REQUEST_CHECKPOINT_STATS_SNAPSHOT; requestJobResultFunction = DEFAULT_REQUEST_JOB_RESULT_FUNCTION; requestJobStatusFunction = DEFAULT_REQUEST_JOB_STATUS_FUNCTION; requestMultipleJobDetailsSupplier = DEFAULT_REQUEST_MULTIPLE_JOB_DETAILS_SUPPLIER; + requestMultipleApplicationDetailsSupplier = + DEFAULT_REQUEST_MULTIPLE_APPLICATION_DETAILS_SUPPLIER; requestClusterOverviewSupplier = DEFAULT_REQUEST_CLUSTER_OVERVIEW_SUPPLIER; requestMetricQueryServiceGatewaysSupplier = DEFAULT_REQUEST_METRIC_QUERY_SERVICE_PATHS_SUPPLIER; @@ -576,6 +637,12 @@ public T setCancelJobFunction( return self(); } + public T setCancelApplicationFunction( + Function> cancelApplicationFunction) { + this.cancelApplicationFunction = cancelApplicationFunction; + return self(); + } + public T setTriggerCheckpointFunction( BiFunction< AsynchronousJobOperationKey, @@ -653,12 +720,15 @@ public TestingRestfulGateway build() { address, hostname, cancelJobFunction, + cancelApplicationFunction, requestJobFunction, + requestApplicationFunction, requestExecutionGraphInfoFunction, requestCheckpointStatsSnapshotFunction, requestJobResultFunction, requestJobStatusFunction, requestMultipleJobDetailsSupplier, + requestMultipleApplicationDetailsSupplier, requestClusterOverviewSupplier, requestMetricQueryServiceGatewaysSupplier, requestTaskManagerMetricQueryServiceGatewaysSupplier, diff --git a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/application/ScriptExecutorITCase.java b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/application/ScriptExecutorITCase.java index 0103c6ec81f6e..4fe639c6c0d28 100644 --- a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/application/ScriptExecutorITCase.java +++ b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/service/application/ScriptExecutorITCase.java @@ -84,7 +84,8 @@ protected String runStatements(List statements) throws Excepti miniCluster.getConfiguration(), ScriptExecutor.class.getClassLoader(), false, - false); + false, + null); executor = new TestScriptExecutor(