/* * 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.event.TaskEvent; import org.apache.flink.runtime.execution.CancelTaskException; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener; import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; import org.apache.flink.runtime.io.network.partition.ProducerFailedException; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Timer; import java.util.TimerTask; import java.util.concurrent.atomic.AtomicLong; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; /** * An input channel, which requests a local subpartition. */ public class LocalInputChannel extends InputChannel implements BufferAvailabilityListener { private static final Logger LOG = LoggerFactory.getLogger(LocalInputChannel.class); // ------------------------------------------------------------------------ private final Object requestLock = new Object(); /** The local partition manager. */ private final ResultPartitionManager partitionManager; /** Task event dispatcher for backwards events. */ private final TaskEventDispatcher taskEventDispatcher; /** Number of available buffers used to keep track of non-empty gate notifications. */ private final AtomicLong numBuffersAvailable; /** The consumed subpartition */ private volatile ResultSubpartitionView subpartitionView; private volatile boolean isReleased; public LocalInputChannel( SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, ResultPartitionManager partitionManager, TaskEventDispatcher taskEventDispatcher, TaskIOMetricGroup metrics) { this(inputGate, channelIndex, partitionId, partitionManager, taskEventDispatcher, 0, 0, metrics); } public LocalInputChannel( SingleInputGate inputGate, int channelIndex, ResultPartitionID partitionId, ResultPartitionManager partitionManager, TaskEventDispatcher taskEventDispatcher, int initialBackoff, int maxBackoff, TaskIOMetricGroup metrics) { super(inputGate, channelIndex, partitionId, initialBackoff, maxBackoff, metrics.getNumBytesInLocalCounter()); this.partitionManager = checkNotNull(partitionManager); this.taskEventDispatcher = checkNotNull(taskEventDispatcher); this.numBuffersAvailable = new AtomicLong(); } // ------------------------------------------------------------------------ // Consume // ------------------------------------------------------------------------ @Override void requestSubpartition(int subpartitionIndex) throws IOException, InterruptedException { boolean retriggerRequest = false; // The lock is required to request only once in the presence of retriggered requests. synchronized (requestLock) { checkState(!isReleased, "LocalInputChannel has been released already"); if (subpartitionView == null) { LOG.debug("{}: Requesting LOCAL subpartition {} of partition {}.", this, subpartitionIndex, partitionId); try { ResultSubpartitionView subpartitionView = partitionManager.createSubpartitionView( partitionId, subpartitionIndex, this); if (subpartitionView == null) { throw new IOException("Error requesting subpartition."); } // make the subpartition view visible this.subpartitionView = subpartitionView; // check if the channel was released in the meantime if (isReleased) { subpartitionView.releaseAllResources(); this.subpartitionView = null; } } catch (PartitionNotFoundException notFound) { if (increaseBackoff()) { retriggerRequest = true; } else { throw notFound; } } } } // Do this outside of the lock scope as this might lead to a // deadlock with a concurrent release of the channel via the // input gate. if (retriggerRequest) { inputGate.retriggerPartitionRequest(partitionId.getPartitionId()); } } /** * Retriggers a subpartition request. */ void retriggerSubpartitionRequest(Timer timer, final int subpartitionIndex) { synchronized (requestLock) { checkState(subpartitionView == null, "already requested partition"); timer.schedule(new TimerTask() { @Override public void run() { try { requestSubpartition(subpartitionIndex); } catch (Throwable t) { setError(t); } } }, getCurrentBackoff()); } } @Override BufferAndAvailability getNextBuffer() throws IOException, InterruptedException { checkError(); ResultSubpartitionView subpartitionView = this.subpartitionView; if (subpartitionView == null) { // this can happen if the request for the partition was triggered asynchronously // by the time trigger // would be good to avoid that, by guaranteeing that the requestPartition() and // getNextBuffer() always come from the same thread // we could do that by letting the timer insert a special "requesting channel" into the input gate's queue subpartitionView = checkAndWaitForSubpartitionView(); } Buffer next = subpartitionView.getNextBuffer(); if (next == null) { if (subpartitionView.isReleased()) { throw new CancelTaskException("Consumed partition " + subpartitionView + " has been released."); } else { // This means there is a bug in the buffer availability // notifications. throw new IllegalStateException("Consumed partition has no buffers available. " + "Number of received buffer notifications is " + numBuffersAvailable + "."); } } long remaining = numBuffersAvailable.decrementAndGet(); if (remaining >= 0) { numBytesIn.inc(next.getSize()); return new BufferAndAvailability(next, remaining > 0); } else if (subpartitionView.isReleased()) { throw new ProducerFailedException(subpartitionView.getFailureCause()); } else { throw new IllegalStateException("No buffer available and producer partition not released."); } } @Override public void notifyBuffersAvailable(long numBuffers) { // if this request made the channel non-empty, notify the input gate if (numBuffers > 0 && numBuffersAvailable.getAndAdd(numBuffers) == 0) { notifyChannelNonEmpty(); } } private ResultSubpartitionView checkAndWaitForSubpartitionView() { // synchronizing on the request lock means this blocks until the asynchronous request // for the partition view has been completed // by then the subpartition view is visible or the channel is released synchronized (requestLock) { checkState(!isReleased, "released"); checkState(subpartitionView != null, "Queried for a buffer before requesting the subpartition."); return subpartitionView; } } // ------------------------------------------------------------------------ // Task events // ------------------------------------------------------------------------ @Override void sendTaskEvent(TaskEvent event) throws IOException { checkError(); checkState(subpartitionView != null, "Tried to send task event to producer before requesting the subpartition."); if (!taskEventDispatcher.publish(partitionId, event)) { throw new IOException("Error while publishing event " + event + " to producer. The producer could not be found."); } } // ------------------------------------------------------------------------ // Life cycle // ------------------------------------------------------------------------ @Override boolean isReleased() { return isReleased; } @Override void notifySubpartitionConsumed() throws IOException { if (subpartitionView != null) { subpartitionView.notifySubpartitionConsumed(); } } /** * Releases the partition reader */ @Override void releaseAllResources() throws IOException { if (!isReleased) { isReleased = true; ResultSubpartitionView view = subpartitionView; if (view != null) { view.releaseAllResources(); subpartitionView = null; } } } @Override public String toString() { return "LocalInputChannel [" + partitionId + "]"; } }