zhuzhurk commented on code in PR #27307: URL: https://github.com/apache/flink/pull/27307#discussion_r2608946126
########## flink-clients/src/main/java/org/apache/flink/client/deployment/application/PackagedProgramApplication.java: ########## @@ -0,0 +1,639 @@ +/* + * 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.ApplicationID; +import org.apache.flink.api.common.ApplicationState; +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.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.PipelineOptionsInternal; +import org.apache.flink.core.execution.PipelineExecutorServiceLoader; +import org.apache.flink.runtime.application.AbstractApplication; +import org.apache.flink.runtime.client.DuplicateJobSubmissionException; +import org.apache.flink.runtime.clusterframework.ApplicationStatus; +import org.apache.flink.runtime.dispatcher.DispatcherGateway; +import org.apache.flink.runtime.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.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.Objects; +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.Executor; +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; + +/** + * An implementation of {@link AbstractApplication} designed for executing the user's {@code + * main()}. + * + * <p>This application functions similarly to {@link ApplicationDispatcherBootstrap} and can serve + * as a replacement for it in application mode. + */ +@Internal +public class PackagedProgramApplication extends AbstractApplication { + + @VisibleForTesting static final String FAILED_JOB_NAME = "(application driver)"; + + private static final Logger LOG = LoggerFactory.getLogger(PackagedProgramApplication.class); + + private static boolean isCanceledOrFailed(ApplicationStatus applicationStatus) { Review Comment: This method should be part of `ApplicationStatus`. ########## flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgram.java: ########## @@ -91,7 +92,9 @@ public class PackagedProgram implements AutoCloseable { private final List<URL> classpaths; - private final URLClassLoader userCodeClassLoader; + private final Configuration configuration; + + private transient URLClassLoader userCodeClassLoader; Review Comment: After ser/de-ser, the `mainClass` is no longer loaded by the `userCodeClassLoader`, which is not as expected. ########## flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ApplicationStatus.java: ########## @@ -89,4 +90,41 @@ public JobStatus deriveJobStatus() { return JOB_STATUS_APPLICATION_STATUS_BI_MAP.inverse().get(this); } + + /** + * A bidirectional mapping between {@code ApplicationState} and {@code ApplicationStatus}. + * + * <p>{@code ApplicationState} covers the entire lifecycle of an application, representing + * various stages from created to finish. + * + * <p>{@code ApplicationStatus}, on the other hand, describes the final state of the cluster + * when shutdown. The class name may be changed in the future after refactoring its usage. + */ + private static final BiMap<ApplicationState, ApplicationStatus> + APPLICATION_STATE_APPLICATION_STATUS_BI_MAP = + EnumBiMap.create(ApplicationState.class, ApplicationStatus.class); + + static { + APPLICATION_STATE_APPLICATION_STATUS_BI_MAP.put( + ApplicationState.FAILED, ApplicationStatus.FAILED); + APPLICATION_STATE_APPLICATION_STATUS_BI_MAP.put( + ApplicationState.CANCELED, ApplicationStatus.CANCELED); + APPLICATION_STATE_APPLICATION_STATUS_BI_MAP.put( + ApplicationState.FINISHED, ApplicationStatus.SUCCEEDED); + } + + /** + * Derives the {@link ApplicationState} from the {@code ApplicationStatus}. + * + * @return The corresponding {@code ApplicationState}. + * @throws UnsupportedOperationException for {@link #UNKNOWN}. + */ + public ApplicationState deriveApplicationState() { Review Comment: Is it possible to derive ApplicationState from JobStatus, instead of introducing the mapping chain JobStatus -> ApplicationStatus -> ApplicationState? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
