eemario commented on code in PR #27306:
URL: https://github.com/apache/flink/pull/27306#discussion_r2618309162


##########
flink-runtime/src/main/java/org/apache/flink/runtime/application/AbstractApplication.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * 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.application;
+
+import org.apache.flink.annotation.Internal;
+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.runtime.dispatcher.DispatcherGateway;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.FatalErrorHandler;
+import org.apache.flink.util.concurrent.ScheduledExecutor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+
+/** Base class for all applications. */
+@Internal
+public abstract class AbstractApplication implements Serializable {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(AbstractApplication.class);
+
+    private static final long serialVersionUID = 1L;
+
+    private final ApplicationID applicationId;
+
+    private ApplicationState applicationState;
+
+    /**
+     * Timestamps (in milliseconds as returned by {@code 
System.currentTimeMillis()}) when the
+     * application transitioned into a certain status. The index into this 
array is the ordinal of
+     * the enum value, i.e. the timestamp when the application went into state 
"RUNNING" is at
+     * {@code timestamps[RUNNING.ordinal()]}.
+     */
+    private final long[] statusTimestamps;
+
+    private final Set<JobID> jobs = new HashSet<>();
+
+    public AbstractApplication(ApplicationID applicationId) {
+        this.applicationId = applicationId;
+        this.statusTimestamps = new long[ApplicationState.values().length];
+        this.applicationState = ApplicationState.CREATED;
+        this.statusTimestamps[ApplicationState.CREATED.ordinal()] = 
System.currentTimeMillis();
+    }
+
+    /**
+     * Entry method to run the application asynchronously.
+     *
+     * <p>The returned CompletableFuture indicates that the execution request 
has been accepted and
+     * the application transitions to RUNNING state.
+     *
+     * <p><b>Note:</b> This method must be called on the main thread.
+     *
+     * @param dispatcherGateway the dispatcher of the cluster to run the 
application.
+     * @param scheduledExecutor the executor to run the user logic.
+     * @param mainThreadExecutor the executor bound to the main thread.
+     * @param errorHandler the handler for fatal errors.
+     * @return a future indicating that the execution request has been 
accepted.
+     */
+    public abstract CompletableFuture<Acknowledge> execute(
+            final DispatcherGateway dispatcherGateway,
+            final ScheduledExecutor scheduledExecutor,
+            final Executor mainThreadExecutor,
+            final FatalErrorHandler errorHandler);
+
+    /**
+     * Cancels the application execution.
+     *
+     * <p>This method is responsible for initiating the cancellation process 
and handling the
+     * appropriate state transitions of the application.
+     *
+     * <p><b>Note:</b> This method must be called from the main thread.
+     */
+    public abstract void cancel();
+
+    /**
+     * Cleans up execution associated with the application.
+     *
+     * <p>This method is typically invoked when the cluster is shutting down.
+     */
+    public abstract void dispose();
+
+    public abstract String getName();
+
+    public ApplicationID getApplicationId() {
+        return applicationId;
+    }
+
+    public Set<JobID> getJobs() {
+        return Collections.unmodifiableSet(jobs);
+    }
+
+    /**
+     * Adds a job ID to the jobs set.
+     *
+     * <p>This method is not thread-safe and should not be called concurrently.
+     */
+    public boolean addJob(JobID jobId) {

Review Comment:
   This method is required to be called in the main thread. Updated the 
comments to remove the ambiguity.



-- 
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]

Reply via email to