|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.flink.runtime.application; |
| 20 | + |
| 21 | +import org.apache.flink.api.common.ApplicationID; |
| 22 | +import org.apache.flink.api.common.ApplicationState; |
| 23 | +import org.apache.flink.api.common.JobID; |
| 24 | +import org.apache.flink.runtime.dispatcher.Dispatcher; |
| 25 | +import org.apache.flink.runtime.dispatcher.DispatcherGateway; |
| 26 | +import org.apache.flink.runtime.messages.Acknowledge; |
| 27 | +import org.apache.flink.runtime.rpc.FatalErrorHandler; |
| 28 | +import org.apache.flink.util.concurrent.ScheduledExecutor; |
| 29 | + |
| 30 | +import org.slf4j.Logger; |
| 31 | +import org.slf4j.LoggerFactory; |
| 32 | + |
| 33 | +import java.io.Serializable; |
| 34 | +import java.util.Arrays; |
| 35 | +import java.util.Collections; |
| 36 | +import java.util.EnumMap; |
| 37 | +import java.util.HashSet; |
| 38 | +import java.util.Map; |
| 39 | +import java.util.Set; |
| 40 | +import java.util.concurrent.CompletableFuture; |
| 41 | +import java.util.concurrent.Executor; |
| 42 | + |
| 43 | +/** Base class for all applications. */ |
| 44 | +public abstract class AbstractApplication implements Serializable { |
| 45 | + |
| 46 | + private static final Logger LOG = LoggerFactory.getLogger(AbstractApplication.class); |
| 47 | + |
| 48 | + private static final long serialVersionUID = 1L; |
| 49 | + |
| 50 | + private final ApplicationID applicationId; |
| 51 | + |
| 52 | + private ApplicationState applicationState; |
| 53 | + |
| 54 | + /** |
| 55 | + * Timestamps (in milliseconds as returned by {@code System.currentTimeMillis()}) when the |
| 56 | + * application transitioned into a certain status. The index into this array is the ordinal of |
| 57 | + * the enum value, i.e. the timestamp when the application went into state "RUNNING" is at |
| 58 | + * {@code timestamps[RUNNING.ordinal()]}. |
| 59 | + */ |
| 60 | + private final long[] statusTimestamps; |
| 61 | + |
| 62 | + private final Set<JobID> jobs = new HashSet<>(); |
| 63 | + |
| 64 | + public AbstractApplication(ApplicationID applicationId) { |
| 65 | + this.applicationId = applicationId; |
| 66 | + this.statusTimestamps = new long[ApplicationState.values().length]; |
| 67 | + this.applicationState = ApplicationState.CREATED; |
| 68 | + this.statusTimestamps[ApplicationState.CREATED.ordinal()] = System.currentTimeMillis(); |
| 69 | + } |
| 70 | + |
| 71 | + /** |
| 72 | + * Entry method to run the application asynchronously. |
| 73 | + * |
| 74 | + * <p>The returned CompletableFuture indicates that the execution request has been accepted and |
| 75 | + * the application transitions to RUNNING state. |
| 76 | + * |
| 77 | + * <p><b>Note:</b> This method must be called in the main thread of the {@link Dispatcher}. |
| 78 | + * |
| 79 | + * @param dispatcherGateway the dispatcher of the cluster to run the application. |
| 80 | + * @param scheduledExecutor the executor to run the user logic. |
| 81 | + * @param mainThreadExecutor the executor bound to the main thread. |
| 82 | + * @param errorHandler the handler for fatal errors. |
| 83 | + * @return a future indicating that the execution request has been accepted. |
| 84 | + */ |
| 85 | + public abstract CompletableFuture<Acknowledge> execute( |
| 86 | + final DispatcherGateway dispatcherGateway, |
| 87 | + final ScheduledExecutor scheduledExecutor, |
| 88 | + final Executor mainThreadExecutor, |
| 89 | + final FatalErrorHandler errorHandler); |
| 90 | + |
| 91 | + /** |
| 92 | + * Cancels the application execution. |
| 93 | + * |
| 94 | + * <p>This method is responsible for initiating the cancellation process and handling the |
| 95 | + * appropriate state transitions of the application. |
| 96 | + * |
| 97 | + * <p><b>Note:</b> This method must be called in the main thread of the {@link Dispatcher}. |
| 98 | + */ |
| 99 | + public abstract void cancel(); |
| 100 | + |
| 101 | + /** |
| 102 | + * Cleans up execution associated with the application. |
| 103 | + * |
| 104 | + * <p>This method is typically invoked when the cluster is shutting down. |
| 105 | + */ |
| 106 | + public abstract void dispose(); |
| 107 | + |
| 108 | + public abstract String getName(); |
| 109 | + |
| 110 | + public ApplicationID getApplicationId() { |
| 111 | + return applicationId; |
| 112 | + } |
| 113 | + |
| 114 | + public Set<JobID> getJobs() { |
| 115 | + return Collections.unmodifiableSet(jobs); |
| 116 | + } |
| 117 | + |
| 118 | + /** |
| 119 | + * Adds a job ID to the jobs set. |
| 120 | + * |
| 121 | + * <p><b>Note:</b>This method must be called in the main thread of the {@link Dispatcher}. |
| 122 | + */ |
| 123 | + public boolean addJob(JobID jobId) { |
| 124 | + return jobs.add(jobId); |
| 125 | + } |
| 126 | + |
| 127 | + public ApplicationState getApplicationStatus() { |
| 128 | + return applicationState; |
| 129 | + } |
| 130 | + |
| 131 | + // ------------------------------------------------------------------------ |
| 132 | + // State Transitions |
| 133 | + // ------------------------------------------------------------------------ |
| 134 | + |
| 135 | + private static final Map<ApplicationState, Set<ApplicationState>> ALLOWED_TRANSITIONS; |
| 136 | + |
| 137 | + static { |
| 138 | + ALLOWED_TRANSITIONS = new EnumMap<>(ApplicationState.class); |
| 139 | + ALLOWED_TRANSITIONS.put( |
| 140 | + ApplicationState.CREATED, |
| 141 | + new HashSet<>(Arrays.asList(ApplicationState.RUNNING, ApplicationState.CANCELING))); |
| 142 | + ALLOWED_TRANSITIONS.put( |
| 143 | + ApplicationState.RUNNING, |
| 144 | + new HashSet<>( |
| 145 | + Arrays.asList( |
| 146 | + ApplicationState.FINISHED, |
| 147 | + ApplicationState.FAILING, |
| 148 | + ApplicationState.CANCELING))); |
| 149 | + ALLOWED_TRANSITIONS.put( |
| 150 | + ApplicationState.FAILING, |
| 151 | + new HashSet<>(Collections.singletonList(ApplicationState.FAILED))); |
| 152 | + ALLOWED_TRANSITIONS.put( |
| 153 | + ApplicationState.CANCELING, |
| 154 | + new HashSet<>(Collections.singletonList(ApplicationState.CANCELED))); |
| 155 | + } |
| 156 | + |
| 157 | + /** All state transition methods must be called in the main thread. */ |
| 158 | + public void transitionToRunning() { |
| 159 | + transitionState(ApplicationState.RUNNING); |
| 160 | + } |
| 161 | + |
| 162 | + /** All state transition methods must be called in the main thread. */ |
| 163 | + public void transitionToCanceling() { |
| 164 | + transitionState(ApplicationState.CANCELING); |
| 165 | + } |
| 166 | + |
| 167 | + /** All state transition methods must be called in the main thread. */ |
| 168 | + public void transitionToFailing() { |
| 169 | + transitionState(ApplicationState.FAILING); |
| 170 | + } |
| 171 | + |
| 172 | + /** All state transition methods must be called in the main thread. */ |
| 173 | + public void transitionToFailed() { |
| 174 | + transitionState(ApplicationState.FAILED); |
| 175 | + } |
| 176 | + |
| 177 | + /** All state transition methods must be called in the main thread. */ |
| 178 | + public void transitionToFinished() { |
| 179 | + transitionState(ApplicationState.FINISHED); |
| 180 | + } |
| 181 | + |
| 182 | + /** All state transition methods must be called in the main thread. */ |
| 183 | + public void transitionToCanceled() { |
| 184 | + transitionState(ApplicationState.CANCELED); |
| 185 | + } |
| 186 | + |
| 187 | + void transitionState(ApplicationState targetState) { |
| 188 | + validateTransition(targetState); |
| 189 | + LOG.info( |
| 190 | + "Application {} ({}) switched from state {} to {}.", |
| 191 | + getName(), |
| 192 | + getApplicationId(), |
| 193 | + applicationState, |
| 194 | + targetState); |
| 195 | + this.statusTimestamps[targetState.ordinal()] = System.currentTimeMillis(); |
| 196 | + this.applicationState = targetState; |
| 197 | + } |
| 198 | + |
| 199 | + private void validateTransition(ApplicationState targetState) { |
| 200 | + Set<ApplicationState> allowedTransitions = ALLOWED_TRANSITIONS.get(applicationState); |
| 201 | + if (allowedTransitions == null || !allowedTransitions.contains(targetState)) { |
| 202 | + throw new IllegalStateException( |
| 203 | + String.format( |
| 204 | + "Invalid transition from %s to %s", applicationState, targetState)); |
| 205 | + } |
| 206 | + } |
| 207 | + |
| 208 | + public long getStatusTimestamp(ApplicationState status) { |
| 209 | + return this.statusTimestamps[status.ordinal()]; |
| 210 | + } |
| 211 | +} |
0 commit comments