tillrohrmann commented on a change in pull request #8463: [FLINK-12530][network] Move Task.inputGatesById to NetworkEnvironment URL: https://github.com/apache/flink/pull/8463#discussion_r288288062
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteChannelStateChecker.java ########## @@ -0,0 +1,125 @@ +/* + * 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.io.network.partition.consumer; + +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.io.network.partition.PartitionStateChecker; +import org.apache.flink.runtime.io.network.partition.PartitionStateChecker.CheckResult; +import org.apache.flink.runtime.io.network.partition.ResultPartitionID; +import org.apache.flink.runtime.jobmanager.PartitionProducerDisposedException; +import org.apache.flink.types.Either; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.TimeoutException; + +/** + * Handles the result of {@link PartitionStateChecker}. + * + * <p>The method {@code isProducerConsumerReadyOrAbortConsumption} determines whether the partition producer is + * in a producing state and consumer is ready for consumption. + */ +public class RemoteChannelStateChecker { + private static final Logger LOG = LoggerFactory.getLogger(RemoteChannelStateChecker.class); + + private final ResultPartitionID resultPartitionId; + + private final String taskNameWithSubtask; + + public RemoteChannelStateChecker(ResultPartitionID resultPartitionId, String taskNameWithSubtask) { + this.resultPartitionId = resultPartitionId; + this.taskNameWithSubtask = taskNameWithSubtask; + } + + public boolean isProducerConsumerReadyOrAbortConsumption(CheckResult checkResult) { + Either<ExecutionState, Throwable> result = checkResult.getProducerExecutionState(); + if (result.isLeft() || result.right() instanceof TimeoutException) { + boolean isProducerConsumerReady = isProducerConsumerReady(checkResult); + if (isProducerConsumerReady) { + return true; + } else { + abortConsumptionOrIgnoreCheckResult(checkResult); + } + } else { + handleFailedCheckResult(checkResult); + } + return false; + } + + private boolean isProducerConsumerReady(CheckResult checkResult) { + ExecutionState consumerState = checkResult.getConsumerExecutionState(); + Either<ExecutionState, Throwable> result = checkResult.getProducerExecutionState(); + ExecutionState producerState = result.isLeft() ? result.left() : ExecutionState.RUNNING; + return consumerState == ExecutionState.RUNNING && + (producerState == ExecutionState.SCHEDULED || + producerState == ExecutionState.DEPLOYING || + producerState == ExecutionState.RUNNING || + producerState == ExecutionState.FINISHED); + } + + private void abortConsumptionOrIgnoreCheckResult(CheckResult checkResult) { + ExecutionState consumerState = checkResult.getConsumerExecutionState(); + Either<ExecutionState, Throwable> result = checkResult.getProducerExecutionState(); + ExecutionState producerState = result.isLeft() ? result.left() : ExecutionState.RUNNING; + if (consumerState == ExecutionState.RUNNING) { + if (producerState == ExecutionState.CANCELING || + producerState == ExecutionState.CANCELED || + producerState == ExecutionState.FAILED) { + + // The producing execution has been canceled or failed. We + // don't need to re-trigger the request since it cannot + // succeed. + if (LOG.isDebugEnabled()) { + LOG.debug("Cancelling task {} after the producer of partition {} with attempt ID {} has entered state {}.", + taskNameWithSubtask, + resultPartitionId.getPartitionId(), + resultPartitionId.getProducerId(), + producerState); + } + + checkResult.cancelConsumption(); + } else { + // Any other execution state is unexpected. Currently, only + // state CREATED is left out of the checked states. If we + // see a producer in this state, something went wrong with + // scheduling in topological order. + final String msg = String.format("Producer with attempt ID %s of partition %s in unexpected state %s.", + resultPartitionId.getProducerId(), + resultPartitionId.getPartitionId(), + producerState); + + checkResult.failConsumption(new IllegalStateException(msg)); + } + } else { Review comment: The else branch would not be needed if we check the consumer state before calling into the `RemoteChannelStateChecker`. ---------------------------------------------------------------- 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: us...@infra.apache.org With regards, Apache Git Services