rmetzger commented on a change in pull request #14879: URL: https://github.com/apache/flink/pull/14879#discussion_r570873065
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/declarative/Executing.java ########## @@ -0,0 +1,278 @@ +/* + * 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.scheduler.declarative; + +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.scheduler.ExecutionGraphHandler; +import org.apache.flink.runtime.scheduler.InternalFailuresListener; +import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; + +import javax.annotation.Nullable; + +import java.time.Duration; + +/** State which represents a running job with an {@link ExecutionGraph} and assigned slots. */ +class Executing extends StateWithExecutionGraph + implements ResourceConsumer, InternalFailuresListener { + + private final Context context; + + private final ClassLoader userCodeClassLoader; + + Executing( + ExecutionGraph executionGraph, + ExecutionGraphHandler executionGraphHandler, + OperatorCoordinatorHandler operatorCoordinatorHandler, + Logger logger, + Context context, + ClassLoader userCodeClassLoader) { + super(context, executionGraph, executionGraphHandler, operatorCoordinatorHandler, logger); + this.context = context; + this.userCodeClassLoader = userCodeClassLoader; + } + + @Override + public void onEnter() { + deploy(); + } + + @Override + public void cancel() { + context.goToCanceling( + getExecutionGraph(), getExecutionGraphHandler(), getOperatorCoordinatorHandler()); + } + + @Override + public void handleGlobalFailure(Throwable cause) { + handleAnyFailure(cause); + } + + @Override + public void notifyTaskFailure( + ExecutionAttemptID attemptId, + Throwable cause, + boolean cancelTask, + boolean releasePartitions) { + handleAnyFailure(cause); + } + + @Override + public void notifyGlobalFailure(Throwable cause) { + handleAnyFailure(cause); + } Review comment: Note to reviewers: I introduced these two methods compared to the POC version. They are there for handling failures during deployment (`InternalFailuresListener` interface) ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/declarative/Executing.java ########## @@ -0,0 +1,278 @@ +/* + * 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.scheduler.declarative; + +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionGraph; +import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.executiongraph.TaskExecutionStateTransition; +import org.apache.flink.runtime.scheduler.ExecutionGraphHandler; +import org.apache.flink.runtime.scheduler.InternalFailuresListener; +import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; + +import javax.annotation.Nullable; + +import java.time.Duration; + +/** State which represents a running job with an {@link ExecutionGraph} and assigned slots. */ +class Executing extends StateWithExecutionGraph + implements ResourceConsumer, InternalFailuresListener { + + private final Context context; + + private final ClassLoader userCodeClassLoader; + + Executing( + ExecutionGraph executionGraph, + ExecutionGraphHandler executionGraphHandler, + OperatorCoordinatorHandler operatorCoordinatorHandler, + Logger logger, + Context context, + ClassLoader userCodeClassLoader) { + super(context, executionGraph, executionGraphHandler, operatorCoordinatorHandler, logger); + this.context = context; + this.userCodeClassLoader = userCodeClassLoader; + } + + @Override + public void onEnter() { + deploy(); + } + + @Override + public void cancel() { + context.goToCanceling( + getExecutionGraph(), getExecutionGraphHandler(), getOperatorCoordinatorHandler()); + } + + @Override + public void handleGlobalFailure(Throwable cause) { + handleAnyFailure(cause); + } + + @Override + public void notifyTaskFailure( + ExecutionAttemptID attemptId, + Throwable cause, + boolean cancelTask, + boolean releasePartitions) { + handleAnyFailure(cause); + } + + @Override + public void notifyGlobalFailure(Throwable cause) { + handleAnyFailure(cause); + } + + private void handleAnyFailure(Throwable cause) { + final FailureResult failureResult = context.howToHandleFailure(cause); + + if (failureResult.canRestart()) { + context.goToRestarting( + getExecutionGraph(), + getExecutionGraphHandler(), + getOperatorCoordinatorHandler(), + failureResult.getBackoffTime()); + } else { + context.goToFailing( + getExecutionGraph(), + getExecutionGraphHandler(), + getOperatorCoordinatorHandler(), + failureResult.getFailureCause()); + } + } + + @Override + boolean updateTaskExecutionState(TaskExecutionStateTransition taskExecutionState) { + final boolean successfulUpdate = getExecutionGraph().updateState(taskExecutionState); + + if (successfulUpdate) { + if (taskExecutionState.getExecutionState() == ExecutionState.FAILED) { + Throwable cause = taskExecutionState.getError(userCodeClassLoader); + handleAnyFailure(cause); + } + } + + return successfulUpdate; + } + + @Override + void onTerminalState(JobStatus terminalState) { + context.goToFinished(ArchivedExecutionGraph.createFrom(getExecutionGraph())); + } + + private void deploy() { + final ExecutionGraph executionGraph = getExecutionGraph(); + executionGraph.setInternalTaskFailuresListener(this); Review comment: Here, I register the failure listener (also a change compared to the POC) ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected]
