From 408a6fae81e9c7739ddc17f4a534eb400ed0657d Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Tue, 6 Jan 2015 17:11:08 +0100 Subject: [PATCH 1/2] [FLINK-1350][FLINK-1359][Distributed runtime] Add blocking result partition variant - Renames runtime intermediate result classes: a) Removes "Intermediate" prefix b) Queue => Subpartition c) Iterator => View - [FLINK-1350] Adds a spillable result subpartition variant for BLOCKING results, which writes data to memory first and starts to spill (asynchronously) if not enough memory is available to produce the result in-memory only. Receiving tasks of BLOCKING results are only deployed after *all* partitions have been fully produced. PIPELINED and BLOCKING results can not be mixed. - [FLINK-1359] Adds simple state tracking to result partitions with notifications after partitions/subpartitions have been consumed. Each partition has to be consumed at least once before it can be released. Currently there is no notion of historic intermediate results, i.e. results are released as soon as they are consumed. --- .../streaming/io/StreamRecordWriter.java | 2 +- .../PartitionDeploymentDescriptor.java | 24 +- .../runtime/deployment/PartitionInfo.java | 14 +- .../runtime/execution/RuntimeEnvironment.java | 14 +- .../runtime/executiongraph/Execution.java | 18 +- .../executiongraph/ExecutionJobVertex.java | 30 +- .../executiongraph/ExecutionVertex.java | 8 +- .../executiongraph/IntermediateResult.java | 25 +- .../IntermediateResultPartition.java | 19 +- .../disk/ChannelReaderInputViewIterator.java | 2 +- .../runtime/io/disk/FileChannelInputView.java | 6 +- .../io/disk/FileChannelOutputView.java | 6 +- .../io/disk/SeekableFileChannelInputView.java | 6 +- .../flink/runtime/io/disk/SpillingBuffer.java | 12 +- .../iomanager/AsynchronousBlockReader.java | 22 +- .../iomanager/AsynchronousBlockWriter.java | 6 +- .../AsynchronousBlockWriterWithCallback.java | 2 +- .../AsynchronousBufferFileReader.java | 48 ++ .../AsynchronousBufferFileSegmentReader.java | 46 ++ .../AsynchronousBufferFileWriter.java | 64 +++ .../iomanager/AsynchronousFileIOChannel.java | 291 +++++++++-- .../io/disk/iomanager/BlockChannelReader.java | 22 +- .../io/disk/iomanager/BlockChannelWriter.java | 10 +- .../BlockChannelWriterWithCallback.java | 12 +- .../io/disk/iomanager/BufferFileReader.java | 33 ++ .../iomanager/BufferFileSegmentReader.java | 31 ++ .../io/disk/iomanager/BufferFileWriter.java | 32 ++ .../iomanager/ChannelReaderInputView.java | 10 +- .../iomanager/ChannelWriterOutputView.java | 8 +- .../io/disk/iomanager/FileSegment.java | 52 ++ .../HeaderlessChannelReaderInputView.java | 4 +- .../runtime/io/disk/iomanager/IOManager.java | 17 +- .../io/disk/iomanager/IOManagerAsync.java | 28 +- .../io/disk/iomanager/QueuingCallback.java | 14 +- .../runtime/io/network/ConnectionManager.java | 5 +- .../io/network/LocalConnectionManager.java | 6 +- .../io/network/NetworkEnvironment.java | 31 +- .../io/network/TaskEventDispatcher.java | 14 +- .../io/network/api/reader/BufferReader.java | 16 +- .../io/network/api/writer/BufferWriter.java | 20 +- .../io/network/api/writer/RecordWriter.java | 4 - .../runtime/io/network/buffer/Buffer.java | 12 +- .../runtime/io/network/buffer/BufferPool.java | 2 +- .../io/network/buffer/BufferPoolOwner.java | 2 +- .../io/network/buffer/LocalBufferPool.java | 12 +- .../io/network/buffer/NetworkBufferPool.java | 8 +- .../network/netty/NettyConnectionManager.java | 8 +- .../io/network/netty/NettyMessage.java | 14 +- .../network/netty/PartitionRequestClient.java | 6 +- .../netty/PartitionRequestClientFactory.java | 2 +- .../netty/PartitionRequestProtocol.java | 11 +- .../network/netty/PartitionRequestQueue.java | 39 +- .../netty/PartitionRequestServerHandler.java | 21 +- ... IllegalSubpartitionRequestException.java} | 8 +- .../IntermediateResultPartition.java | 307 ----------- .../IntermediateResultPartitionManager.java | 131 ----- ...nQueue.java => PipelinedSubpartition.java} | 51 +- .../io/network/partition/ResultPartition.java | 482 ++++++++++++++++++ .../partition/ResultPartitionManager.java | 113 ++++ ...ider.java => ResultPartitionProvider.java} | 14 +- .../partition/ResultPartitionType.java} | 13 +- .../network/partition/ResultSubpartition.java | 67 +++ ...rator.java => ResultSubpartitionView.java} | 16 +- .../partition/SpillableSubpartition.java | 254 +++++++++ .../partition/SpilledSubpartitionView.java | 194 +++++++ .../partition/consumer/InputChannel.java | 12 +- .../partition/consumer/LocalInputChannel.java | 22 +- .../consumer/RemoteInputChannel.java | 4 +- .../consumer/UnknownInputChannel.java | 4 +- .../IntermediateResultPartitionQueue.java | 53 -- .../iterative/io/SerializedUpdateBuffer.java | 8 +- .../runtime/jobgraph/AbstractJobVertex.java | 7 +- .../runtime/jobgraph/IntermediateDataSet.java | 24 +- ...artitionID.java => ResultPartitionID.java} | 10 +- .../runtime/operators/hash/HashPartition.java | 12 +- .../operators/hash/MutableHashTable.java | 4 +- .../hash/ReOpenableHashPartition.java | 2 +- .../sort/CombiningUnilateralSortMerger.java | 4 +- .../operators/sort/UnilateralSortMerger.java | 6 +- .../flink/runtime/taskmanager/Task.java | 4 +- .../runtime/messages/JobmanagerMessages.scala | 4 +- .../ExecutionGraphDeploymentTest.java | 4 +- .../runtime/io/disk/ChannelViewsTest.java | 24 +- .../AsynchronousBufferFileWriterTest.java | 178 +++++++ ...BufferFileWriterFileSegmentReaderTest.java | 198 +++++++ .../iomanager/BufferFileWriterReaderTest.java | 225 ++++++++ .../io/disk/iomanager/IOManagerAsyncTest.java | 31 +- .../io/disk/iomanager/IOManagerITCase.java | 6 +- .../IOManagerPerformanceBenchmark.java | 4 +- .../io/disk/iomanager/IOManagerTest.java | 23 +- .../network/api/reader/MockBufferReader.java | 4 +- .../network/buffer/BufferPoolFactoryTest.java | 2 +- .../runtime/io/network/buffer/BufferTest.java | 2 +- .../network/buffer/LocalBufferPoolTest.java | 8 +- .../netty/NettyMessageSerializationTest.java | 6 +- .../network/partition/MockBufferFactory.java | 87 ++++ .../network/partition/MockBufferProvider.java | 117 +++++ .../io/network/partition/MockConsumer.java | 7 +- .../partition/MockNotificationListener.java | 13 +- .../io/network/partition/MockProducer.java | 11 +- ...st.java => PipelinedSubpartitionTest.java} | 35 +- .../partition/SpillablePartitionTest.java | 125 +++++ .../partition/SpilledPartitionViewTest.java | 243 +++++++++ .../runtime/taskmanager/TaskManagerTest.java | 12 +- .../exampleJavaPrograms/WordCountITCase.java | 6 +- 105 files changed, 3401 insertions(+), 971 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileReader.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileSegmentReader.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriter.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileReader.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileSegmentReader.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriter.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/FileSegment.java rename flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/{queue/IllegalQueueIteratorRequestException.java => IllegalSubpartitionRequestException.java} (79%) delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartition.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartitionManager.java rename flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/{queue/PipelinedPartitionQueue.java => PipelinedSubpartition.java} (72%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java rename flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/{IntermediateResultPartitionProvider.java => ResultPartitionProvider.java} (68%) rename flink-runtime/src/main/java/org/apache/flink/runtime/{jobgraph/IntermediateResultPartitionType.java => io/network/partition/ResultPartitionType.java} (74%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java rename flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/{queue/IntermediateResultPartitionQueueIterator.java => ResultSubpartitionView.java} (91%) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IntermediateResultPartitionQueue.java rename flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/{IntermediateResultPartitionID.java => ResultPartitionID.java} (79%) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockBufferFactory.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockBufferProvider.java rename flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/{queue/PipelinedPartitionQueueTest.java => PipelinedSubpartitionTest.java} (82%) create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillablePartitionTest.java create mode 100644 flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledPartitionViewTest.java diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamRecordWriter.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamRecordWriter.java index b7af589d8d1c9..5757e782d8841 100755 --- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamRecordWriter.java +++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/io/StreamRecordWriter.java @@ -70,7 +70,7 @@ public void terminate() { } @Override public void run() { - while (running && !writer.isFinished()) { + while (running) { try { flush(); Thread.sleep(timeout); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionDeploymentDescriptor.java index 148f8d4c5d910..62b549cd4d2e0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionDeploymentDescriptor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionDeploymentDescriptor.java @@ -23,8 +23,8 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionType; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import java.io.IOException; import java.io.Serializable; @@ -37,19 +37,19 @@ public class PartitionDeploymentDescriptor implements IOReadableWritable, Serial private final IntermediateDataSetID resultId; - private final IntermediateResultPartitionID partitionId; + private final ResultPartitionID partitionId; - private IntermediateResultPartitionType partitionType; + private ResultPartitionType partitionType; private int numberOfQueues; public PartitionDeploymentDescriptor() { this.resultId = new IntermediateDataSetID(); - this.partitionId = new IntermediateResultPartitionID(); + this.partitionId = new ResultPartitionID(); this.numberOfQueues = -1; } - public PartitionDeploymentDescriptor(IntermediateDataSetID resultId, IntermediateResultPartitionID partitionId, IntermediateResultPartitionType partitionType, int numberOfQueues) { + public PartitionDeploymentDescriptor(IntermediateDataSetID resultId, ResultPartitionID partitionId, ResultPartitionType partitionType, int numberOfQueues) { this.resultId = resultId; this.partitionId = partitionId; this.partitionType = partitionType; @@ -64,15 +64,15 @@ public IntermediateDataSetID getResultId() { return resultId; } - public IntermediateResultPartitionID getPartitionId() { + public ResultPartitionID getPartitionId() { return partitionId; } - public IntermediateResultPartitionType getPartitionType() { + public ResultPartitionType getPartitionType() { return partitionType; } - public int getNumberOfQueues() { + public int getNumberOfSubpartitions() { return numberOfQueues; } @@ -92,7 +92,7 @@ public void write(DataOutputView out) throws IOException { public void read(DataInputView in) throws IOException { resultId.read(in); partitionId.read(in); - partitionType = IntermediateResultPartitionType.values()[in.readInt()]; + partitionType = ResultPartitionType.values()[in.readInt()]; numberOfQueues = in.readInt(); } @@ -100,7 +100,7 @@ public void read(DataInputView in) throws IOException { public static PartitionDeploymentDescriptor fromIntermediateResultPartition(IntermediateResultPartition partition) { - IntermediateResultPartitionID partitionId = partition.getPartitionId(); + ResultPartitionID partitionId = partition.getPartitionId(); // The produced data is partitioned at runtime among a number of queues. // If no consumers are known at this point, we use a single queue, @@ -111,6 +111,6 @@ public static PartitionDeploymentDescriptor fromIntermediateResultPartition(Inte numberOfQueues = partition.getConsumers().get(0).size(); } - return new PartitionDeploymentDescriptor(partition.getIntermediateResult().getId(), partitionId, partition.getIntermediateResult().getResultType(), numberOfQueues); + return new PartitionDeploymentDescriptor(partition.getIntermediateResult().getId(), partitionId, partition.getIntermediateResult().getRuntimeType(), numberOfQueues); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionInfo.java index cdaf2895f63f5..515d153132e23 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/PartitionInfo.java @@ -28,7 +28,7 @@ import org.apache.flink.runtime.executiongraph.IntermediateResultPartition; import org.apache.flink.runtime.instance.AllocatedSlot; import org.apache.flink.runtime.io.network.RemoteAddress; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; import java.io.IOException; import java.io.Serializable; @@ -45,7 +45,7 @@ public enum PartitionLocation { LOCAL, REMOTE, UNKNOWN } - private final IntermediateResultPartitionID partitionId; + private final ResultPartitionID partitionId; private ExecutionAttemptID producerExecutionId; @@ -53,7 +53,7 @@ public enum PartitionLocation { private RemoteAddress producerAddress; // != null, iff known remote producer - public PartitionInfo(IntermediateResultPartitionID partitionId, ExecutionAttemptID producerExecutionId, PartitionLocation producerLocation, RemoteAddress producerAddress) { + public PartitionInfo(ResultPartitionID partitionId, ExecutionAttemptID producerExecutionId, PartitionLocation producerLocation, RemoteAddress producerAddress) { this.partitionId = checkNotNull(partitionId); this.producerExecutionId = checkNotNull(producerExecutionId); this.producerLocation = checkNotNull(producerLocation); @@ -61,7 +61,7 @@ public PartitionInfo(IntermediateResultPartitionID partitionId, ExecutionAttempt } public PartitionInfo() { - this.partitionId = new IntermediateResultPartitionID(); + this.partitionId = new ResultPartitionID(); this.producerExecutionId = new ExecutionAttemptID(); this.producerLocation = PartitionLocation.UNKNOWN; this.producerAddress = null; @@ -71,7 +71,7 @@ public PartitionInfo() { // Properties // ------------------------------------------------------------------------ - public IntermediateResultPartitionID getPartitionId() { + public ResultPartitionID getPartitionId() { return partitionId; } @@ -116,7 +116,7 @@ public void read(DataInputView in) throws IOException { public static PartitionInfo fromEdge(ExecutionEdge edge, AllocatedSlot consumerSlot) { IntermediateResultPartition partition = edge.getSource(); - IntermediateResultPartitionID partitionId = partition.getPartitionId(); + ResultPartitionID partitionId = partition.getPartitionId(); // Intermediate result partition producer Execution producer = partition.getProducer().getCurrentExecutionAttempt(); @@ -130,7 +130,7 @@ public static PartitionInfo fromEdge(ExecutionEdge edge, AllocatedSlot consumerS // The producer needs to be running, otherwise the consumer might request a partition, // which has not been registered yet. - if (producerSlot != null && producerState == ExecutionState.RUNNING) { + if (producerSlot != null && (producerState == ExecutionState.RUNNING || producerState == ExecutionState.FINISHED)) { if (producerSlot.getInstance().equals(consumerSlot.getInstance())) { producerLocation = PartitionLocation.LOCAL; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java index 6be03977d742d..c9686a23bfb57 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java @@ -29,7 +29,7 @@ import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.io.network.api.reader.BufferReader; import org.apache.flink.runtime.io.network.api.writer.BufferWriter; -import org.apache.flink.runtime.io.network.partition.IntermediateResultPartition; +import org.apache.flink.runtime.io.network.partition.ResultPartition; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -91,7 +91,7 @@ public class RuntimeEnvironment implements Environment, Runnable { private final AtomicBoolean canceled = new AtomicBoolean(); - private final IntermediateResultPartition[] producedPartitions; + private final ResultPartition[] producedPartitions; private final BufferWriter[] writers; @@ -117,11 +117,13 @@ public RuntimeEnvironment( // Produced intermediate result partitions final List partitions = tdd.getProducedPartitions(); - this.producedPartitions = new IntermediateResultPartition[partitions.size()]; + this.producedPartitions = new ResultPartition[partitions.size()]; this.writers = new BufferWriter[partitions.size()]; for (int i = 0; i < this.producedPartitions.length; i++) { - this.producedPartitions[i] = IntermediateResultPartition.create(this, i, owner.getJobID(), owner.getExecutionId(), networkEnvironment, partitions.get(i)); + PartitionDeploymentDescriptor desc = partitions.get(i); + this.producedPartitions[i] = new ResultPartition(i, owner.getJobID(), owner.getExecutionId(), desc.getPartitionId(), desc.getPartitionType(), desc.getNumberOfSubpartitions(), networkEnvironment, ioManager); + writers[i] = new BufferWriter(this.producedPartitions[i]); } @@ -210,7 +212,7 @@ public void run() { // Finish the produced partitions if (producedPartitions != null) { - for (IntermediateResultPartition partition : producedPartitions) { + for (ResultPartition partition : producedPartitions) { if (partition != null) { partition.finish(); } @@ -364,7 +366,7 @@ public BufferReader[] getAllReaders() { return readers; } - public IntermediateResultPartition[] getProducedPartitions() { + public ResultPartition[] getProducedPartitions() { return producedPartitions; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index a705231caf7f7..e2163c655cca8 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -396,7 +396,7 @@ boolean scheduleOrUpdateConsumers(List> consumers) throws Ex final ExecutionState consumerState = consumerVertex.getExecutionState(); if (consumerState == CREATED) { - if (state == RUNNING) { + if (state == RUNNING || state == FINISHED) { if (!consumerVertex.scheduleForExecution(consumerVertex.getExecutionGraph().getScheduler(), false)) { success = false; } @@ -414,7 +414,6 @@ else if (consumerState == RUNNING) { if (!sendUpdateTaskRpcCall(consumerSlot, consumerExecutionId, edge.getSource().getIntermediateResult().getId(), partitionInfo)) { success = false; } - } else if (consumerState == SCHEDULED || consumerState == DEPLOYING) { success = false; @@ -458,6 +457,17 @@ void markFinished() { if (current == RUNNING || current == DEPLOYING) { if (transitionState(current, FINISHED)) { + for (IntermediateResultPartition partition : vertex.resultPartitions) { + if (!partition.getIntermediateResult().getRuntimeType().isPipelined()) { + try { + scheduleOrUpdateConsumers(partition.getConsumers()); + } + catch (Exception e) { + markFailed(new Exception("Error while scheduleOrUpdateConsumers in markFinished")); + } + } + } + try { assignedResource.releaseSlot(); vertex.getExecutionGraph().deregisterExecution(this); @@ -465,6 +475,7 @@ void markFinished() { finally { vertex.executionFinished(); } + return; } } @@ -594,6 +605,9 @@ private boolean processFail(Throwable t, boolean isCallback) { private boolean switchToRunning() { if (transitionState(DEPLOYING, RUNNING)) { + + // + return true; } else { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java index 7eaa1e660eb44..4c69888843e06 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java @@ -18,15 +18,11 @@ package org.apache.flink.runtime.executiongraph; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - import org.apache.flink.core.io.InputSplit; import org.apache.flink.core.io.InputSplitAssigner; import org.apache.flink.core.io.InputSplitSource; import org.apache.flink.runtime.JobException; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.AbstractJobVertex; import org.apache.flink.runtime.jobgraph.IntermediateDataSet; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; @@ -34,12 +30,17 @@ import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; -import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException; +import org.apache.flink.runtime.jobmanager.scheduler.Scheduler; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.slf4j.Logger; import scala.concurrent.duration.FiniteDuration; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + public class ExecutionJobVertex implements Serializable { static final long serialVersionUID = 42L; @@ -109,9 +110,22 @@ public ExecutionJobVertex(ExecutionGraph graph, AbstractJobVertex jobVertex, // create the intermediate results this.producedDataSets = new IntermediateResult[jobVertex.getNumberOfProducedIntermediateDataSets()]; + + ResultPartitionType type = null; + for (int i = 0; i < jobVertex.getProducedDataSets().size(); i++) { IntermediateDataSet set = jobVertex.getProducedDataSets().get(i); - this.producedDataSets[i] = new IntermediateResult(set.getId(), this, numTaskVertices); + + if (type == null) { + type = set.getRuntimeType(); + } + else if (type != set.getRuntimeType()) { + // TODO This is currently necessary, because otherwise mixing pipelined and blocking + // results will result in the early deployment of blocking results. + throw new IllegalArgumentException("All produced intermediate results must be of the same type."); + } + + this.producedDataSets[i] = new IntermediateResult(set.getId(), this, numTaskVertices, set.getRuntimeType()); } // create all task vertices @@ -123,7 +137,7 @@ public ExecutionJobVertex(ExecutionGraph graph, AbstractJobVertex jobVertex, // sanity check for the double referencing between intermediate result partitions and execution vertices for (IntermediateResult ir : this.producedDataSets) { if (ir.getNumberOfAssignedPartitions() != parallelism) { - throw new RuntimeException("The intermediate result's partitions were not correctly assiged."); + throw new RuntimeException("The intermediate result's partitions were not correctly assigned."); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index a9a5434bdd165..005890053d1bf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -67,7 +67,7 @@ public class ExecutionVertex implements Serializable { private final ExecutionJobVertex jobVertex; - private transient final IntermediateResultPartition[] resultPartitions; + transient final IntermediateResultPartition[] resultPartitions; private transient final ExecutionEdge[][] inputEdges; @@ -367,7 +367,11 @@ boolean scheduleOrUpdateConsumers(int partitionIndex) throws Exception { IntermediateResultPartition partition = resultPartitions[partitionIndex]; - return currentExecution.scheduleOrUpdateConsumers(partition.getConsumers()); + if (partition.finish()) { + return currentExecution.scheduleOrUpdateConsumers(partition.getConsumers()); + } + + return true; } // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java index 64ad2d2a90356..74c4876f5453c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java @@ -19,7 +19,9 @@ package org.apache.flink.runtime.executiongraph; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionType; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; + +import java.util.concurrent.atomic.AtomicInteger; public class IntermediateResult { @@ -37,9 +39,13 @@ public class IntermediateResult { private final int connectionIndex; - private final IntermediateResultPartitionType resultType; + private final ResultPartitionType runtimeType; + + // Counter to keep track of how many of the parallel producers have entered the finished state + // for this this. + private final AtomicInteger runtimeCounter; - public IntermediateResult(IntermediateDataSetID id, ExecutionJobVertex producer, int numParallelProducers) { + public IntermediateResult(IntermediateDataSetID id, ExecutionJobVertex producer, int numParallelProducers, ResultPartitionType runtimeType) { this.id = id; this.producer = producer; this.partitions = new IntermediateResultPartition[numParallelProducers]; @@ -52,8 +58,9 @@ public IntermediateResult(IntermediateDataSetID id, ExecutionJobVertex producer, this.connectionIndex = (int) (Math.random() * Integer.MAX_VALUE); // The runtime type for this produced result - // TODO The JobGraph generator has to decide which type of result this is - this.resultType = IntermediateResultPartitionType.PIPELINED; + this.runtimeType = runtimeType; + + this.runtimeCounter = new AtomicInteger(numParallelProducers); } public void setPartition(int partitionNumber, IntermediateResultPartition partition) { @@ -85,8 +92,8 @@ public int getNumberOfAssignedPartitions() { return partitionsAssigned; } - public IntermediateResultPartitionType getResultType() { - return resultType; + public ResultPartitionType getRuntimeType() { + return runtimeType; } public int registerConsumer() { @@ -104,4 +111,8 @@ public int registerConsumer() { public int getConnectionIndex() { return connectionIndex; } + + int decrementAndGetCounter() { + return runtimeCounter.decrementAndGet(); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java index 7d06dca1cc663..f6624b3652778 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.executiongraph; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; import java.util.ArrayList; import java.util.List; @@ -31,7 +31,7 @@ public class IntermediateResultPartition { private final int partitionNumber; - private final IntermediateResultPartitionID partitionId; + private final ResultPartitionID partitionId; private List> consumers; @@ -40,7 +40,7 @@ public IntermediateResultPartition(IntermediateResult totalResult, ExecutionVert this.producer = producer; this.partitionNumber = partitionNumber; this.consumers = new ArrayList>(0); - this.partitionId = new IntermediateResultPartitionID(); + this.partitionId = new ResultPartitionID(); } public ExecutionVertex getProducer() { @@ -55,7 +55,7 @@ public IntermediateResult getIntermediateResult() { return totalResult; } - public IntermediateResultPartitionID getPartitionId() { + public ResultPartitionID getPartitionId() { return partitionId; } @@ -78,4 +78,15 @@ int addConsumerGroup() { void addConsumer(ExecutionEdge edge, int consumerNumber) { consumers.get(consumerNumber).add(edge); } + + boolean finish() { + if (totalResult.getRuntimeType().isPipelined()) { + return true; + } + else if (totalResult.decrementAndGetCounter() <= 0) { + return true; + } + + return false; + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/ChannelReaderInputViewIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/ChannelReaderInputViewIterator.java index 6007db913b636..87a3baa302d07 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/ChannelReaderInputViewIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/ChannelReaderInputViewIterator.java @@ -61,7 +61,7 @@ public ChannelReaderInputViewIterator(IOManager ioAccess, FileIOChannel.ID chann segments, freeMemTarget, accessors, numBlocks); } - public ChannelReaderInputViewIterator(BlockChannelReader reader, LinkedBlockingQueue returnQueue, + public ChannelReaderInputViewIterator(BlockChannelReader reader, LinkedBlockingQueue returnQueue, List segments, List freeMemTarget, TypeSerializer accessors, int numBlocks) throws IOException { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java index 9fb8072e19cd6..736c245e4c42a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java @@ -39,7 +39,7 @@ */ public class FileChannelInputView extends AbstractPagedInputView { - private final BlockChannelReader reader; + private final BlockChannelReader reader; private final MemoryManager memManager; @@ -53,7 +53,7 @@ public class FileChannelInputView extends AbstractPagedInputView { // -------------------------------------------------------------------------------------------- - public FileChannelInputView(BlockChannelReader reader, MemoryManager memManager, List memory, int sizeOfLastBlock) throws IOException { + public FileChannelInputView(BlockChannelReader reader, MemoryManager memManager, List memory, int sizeOfLastBlock) throws IOException { super(0); checkNotNull(reader); @@ -129,7 +129,7 @@ protected MemorySegment nextSegment(MemorySegment current) throws IOException { // get the next segment numBlocksRemaining--; - return reader.getNextReturnedSegment(); + return reader.getNextReturnedBlock(); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java index e04759cb4a8e3..b6c500f9c58d1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java @@ -35,7 +35,7 @@ */ public class FileChannelOutputView extends AbstractPagedOutputView { - private final BlockChannelWriter writer; // the writer to the channel + private final BlockChannelWriter writer; // the writer to the channel private final MemoryManager memManager; @@ -47,7 +47,7 @@ public class FileChannelOutputView extends AbstractPagedOutputView { // -------------------------------------------------------------------------------------------- - public FileChannelOutputView(BlockChannelWriter writer, MemoryManager memManager, List memory, int segmentSize) throws IOException { + public FileChannelOutputView(BlockChannelWriter writer, MemoryManager memManager, List memory, int segmentSize) throws IOException { super(segmentSize, 0); checkNotNull(writer); @@ -137,7 +137,7 @@ protected MemorySegment nextSegment(MemorySegment current, int posInSegment) thr if (current != null) { writeSegment(current, posInSegment); } - return writer.getNextReturnedSegment(); + return writer.getNextReturnedBlock(); } private void writeSegment(MemorySegment segment, int writePosition) throws IOException { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java index 6098fdb47c002..7d8d48561510d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java @@ -40,7 +40,7 @@ */ public class SeekableFileChannelInputView extends AbstractPagedInputView { - private BlockChannelReader reader; + private BlockChannelReader reader; private final IOManager ioManager; @@ -127,7 +127,7 @@ public void seek(long position) throws IOException { } numBlocksRemaining--; - seekInput(reader.getNextReturnedSegment(), positionInBlock, numBlocksRemaining == 0 ? sizeOfLastBlock : segmentSize); + seekInput(reader.getNextReturnedBlock(), positionInBlock, numBlocksRemaining == 0 ? sizeOfLastBlock : segmentSize); } public void close() throws IOException { @@ -169,7 +169,7 @@ protected MemorySegment nextSegment(MemorySegment current) throws IOException { // get the next segment numBlocksRemaining--; - return reader.getNextReturnedSegment(); + return reader.getNextReturnedBlock(); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java index 655a574ec9dda..5f9c2cffed644 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java @@ -42,7 +42,7 @@ public class SpillingBuffer extends AbstractPagedOutputView { private final MemorySegmentSource memorySource; - private BlockChannelWriter writer; + private BlockChannelWriter writer; private RandomAccessInputView inMemInView; @@ -86,7 +86,7 @@ protected MemorySegment nextSegment(MemorySegment current, int positionInCurrent this.writer.writeBlock(this.fullSegments.get(i)); } this.fullSegments.clear(); - final MemorySegment seg = this.writer.getNextReturnedSegment(); + final MemorySegment seg = this.writer.getNextReturnedBlock(); this.numMemorySegmentsInWriter--; return seg; } @@ -94,7 +94,7 @@ protected MemorySegment nextSegment(MemorySegment current, int positionInCurrent // spilling this.writer.writeBlock(current); this.blockCount++; - return this.writer.getNextReturnedSegment(); + return this.writer.getNextReturnedBlock(); } } @@ -116,7 +116,7 @@ public DataInputView flip() throws IOException { this.blockCount++; this.writer.close(); for (int i = this.numMemorySegmentsInWriter; i > 0; i--) { - this.fullSegments.add(this.writer.getNextReturnedSegment()); + this.fullSegments.add(this.writer.getNextReturnedBlock()); } this.numMemorySegmentsInWriter = 0; } @@ -135,7 +135,7 @@ public DataInputView flip() throws IOException { this.externalInView.close(); } - final BlockChannelReader reader = this.ioManager.createBlockChannelReader(this.writer.getChannelID()); + final BlockChannelReader reader = this.ioManager.createBlockChannelReader(this.writer.getChannelID()); this.externalInView = new HeaderlessChannelReaderInputView(reader, this.fullSegments, this.blockCount, this.numBytesInLastSegment, false); return this.externalInView; } @@ -161,7 +161,7 @@ public List close() throws IOException { // closing before the first flip, collect the memory in the writer this.writer.close(); for (int i = this.numMemorySegmentsInWriter; i > 0; i--) { - segments.add(this.writer.getNextReturnedSegment()); + segments.add(this.writer.getNextReturnedBlock()); } this.writer.closeAndDelete(); this.writer = null; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockReader.java index acfa71f116d75..7a80b7c8b85e0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockReader.java @@ -18,12 +18,12 @@ package org.apache.flink.runtime.io.disk.iomanager; +import org.apache.flink.core.memory.MemorySegment; + import java.io.IOException; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; -import org.apache.flink.core.memory.MemorySegment; - /** * A reader that reads data in blocks from a file channel. The reader reads the blocks into a * {@link org.apache.flink.core.memory.MemorySegment} in an asynchronous fashion. That is, a read @@ -40,7 +40,7 @@ * or even whether the file was written in blocks of the same size, or in blocks at all. Ensuring that the * writing and reading is consistent with each other (same blocks sizes) is up to the programmer. */ -public class AsynchronousBlockReader extends AsynchronousFileIOChannel implements BlockChannelReader { +public class AsynchronousBlockReader extends AsynchronousFileIOChannel implements BlockChannelReader { private final LinkedBlockingQueue returnSegments; @@ -57,7 +57,7 @@ protected AsynchronousBlockReader(FileIOChannel.ID channelID, RequestQueue returnSegments) throws IOException { - super(channelID, requestQueue, new QueuingCallback(returnSegments), false); + super(channelID, requestQueue, new QueuingCallback(returnSegments), false); this.returnSegments = returnSegments; } @@ -74,7 +74,12 @@ protected AsynchronousBlockReader(FileIOChannel.ID channelID, RequestQueue getReturnQueue() { return this.returnSegments; } - - @Override - public void seekToPosition(long position) throws IOException { - this.requestQueue.add(new SeekRequest(this, position)); - } } \ No newline at end of file diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockWriter.java index 7e1681fbed0bf..18d16a09e74e4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBlockWriter.java @@ -24,7 +24,7 @@ import org.apache.flink.core.memory.MemorySegment; -public class AsynchronousBlockWriter extends AsynchronousBlockWriterWithCallback implements BlockChannelWriter { +public class AsynchronousBlockWriter extends AsynchronousBlockWriterWithCallback implements BlockChannelWriter { private final LinkedBlockingQueue returnSegments; @@ -41,7 +41,7 @@ protected AsynchronousBlockWriter(FileIOChannel.ID channelID, RequestQueue returnSegments) throws IOException { - super(channelID, requestQueue, new QueuingCallback(returnSegments)); + super(channelID, requestQueue, new QueuingCallback(returnSegments)); this.returnSegments = returnSegments; } @@ -58,7 +58,7 @@ protected AsynchronousBlockWriter(FileIOChannel.ID channelID, RequestQueue implements BlockChannelWriterWithCallback { +public class AsynchronousBlockWriterWithCallback extends AsynchronousFileIOChannel implements BlockChannelWriterWithCallback { /** * Creates a new asynchronous block writer for the given channel. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileReader.java new file mode 100644 index 0000000000000..0a26962db64c7 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileReader.java @@ -0,0 +1,48 @@ +/* + * 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.disk.iomanager; + +import org.apache.flink.runtime.io.network.buffer.Buffer; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; + +public class AsynchronousBufferFileReader extends AsynchronousFileIOChannel implements BufferFileReader { + + private final AtomicBoolean isConsumed = new AtomicBoolean(); + + protected AsynchronousBufferFileReader(ID channelID, RequestQueue requestQueue, RequestDoneCallback callback) throws IOException { + super(channelID, requestQueue, callback, false); + } + + @Override + public void readInto(Buffer buffer) throws IOException { + addRequest(new BufferReadRequest(this, buffer, isConsumed)); + } + + @Override + public void seekToPosition(long position) throws IOException { + requestQueue.add(new SeekRequest(this, position)); + } + + @Override + public boolean isConsumed() { + return isConsumed.get(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileSegmentReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileSegmentReader.java new file mode 100644 index 0000000000000..145f8a68194f0 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileSegmentReader.java @@ -0,0 +1,46 @@ +/* + * 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.disk.iomanager; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; + +public class AsynchronousBufferFileSegmentReader extends AsynchronousFileIOChannel implements BufferFileSegmentReader { + + private final AtomicBoolean isConsumed = new AtomicBoolean(); + + protected AsynchronousBufferFileSegmentReader(ID channelID, RequestQueue requestQueue, RequestDoneCallback callback) throws IOException { + super(channelID, requestQueue, callback, false); + } + + @Override + public void read() throws IOException { + addRequest(new FileSegmentReadRequest(this, isConsumed)); + } + + @Override + public void seekTo(long position) throws IOException { + requestQueue.add(new SeekRequest(this, position)); + } + + @Override + public boolean isConsumed() { + return isConsumed.get(); + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriter.java new file mode 100644 index 0000000000000..c694c2a8c1f2f --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriter.java @@ -0,0 +1,64 @@ +/* + * 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.disk.iomanager; + +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.util.event.NotificationListener; + +import java.io.IOException; + +public class AsynchronousBufferFileWriter extends AsynchronousFileIOChannel implements BufferFileWriter { + + private static final RecyclingCallback CALLBACK = new RecyclingCallback(); + + protected AsynchronousBufferFileWriter(ID channelID, RequestQueue requestQueue) throws IOException { + super(channelID, requestQueue, CALLBACK, true); + } + + @Override + public void writeBlock(Buffer buffer) throws IOException { + addRequest(new BufferWriteRequest(this, buffer)); + } + + @Override + public int getNumberOfOutstandingRequests() { + return requestsNotReturned.get(); + } + + @Override + public boolean subscribe(NotificationListener listener) throws IOException { + return super.subscribe(listener); + } + + /** + * Recycles the buffer after the I/O request. + */ + private static class RecyclingCallback implements RequestDoneCallback { + + @Override + public void requestSuccessful(Buffer buffer) { + buffer.recycle(); + } + + @Override + public void requestFailed(Buffer buffer, IOException e) { + buffer.recycle(); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannel.java index 281eaadfae63d..cb7abef3ca0a0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannel.java @@ -19,10 +19,14 @@ package org.apache.flink.runtime.io.disk.iomanager; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; +import org.apache.flink.runtime.util.event.NotificationListener; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import static com.google.common.base.Preconditions.checkNotNull; @@ -33,67 +37,73 @@ * segment of the block is added to a collection to be returned. *

* The asynchrony of the access makes it possible to implement read-ahead or write-behind types of I/O accesses. - * + * * @param The type of request (e.g. ReadRequest or WriteRequest issued by this access to the I/O threads. */ public abstract class AsynchronousFileIOChannel extends AbstractFileIOChannel { - - /** The lock that is used during closing to synchronize the thread that waits for all - * requests to be handled with the asynchronous I/O thread. */ + + private final Object subscribeLock = new Object(); + + /** + * The lock that is used during closing to synchronize the thread that waits for all + * requests to be handled with the asynchronous I/O thread. + */ protected final Object closeLock = new Object(); - + /** A request queue for submitting asynchronous requests to the corresponding IO worker thread. */ protected final RequestQueue requestQueue; - + /** An atomic integer that counts the number of requests that we still wait for to return. */ protected final AtomicInteger requestsNotReturned = new AtomicInteger(0); - + /** Hander for completed requests */ protected final RequestDoneCallback resultHandler; - - /** An exception that was encountered by the asynchronous request handling thread.*/ + + /** An exception that was encountered by the asynchronous request handling thread. */ protected volatile IOException exception; - + /** Flag marking this channel as closed */ protected volatile boolean closed; + private volatile NotificationListener allRequestsProcessedListener; + // -------------------------------------------------------------------------------------------- - + /** * Creates a new channel access to the path indicated by the given ID. The channel accepts buffers to be - * read/written and hands them to the asynchronous I/O thread. After being processed, the buffers + * read/written and hands them to the asynchronous I/O thread. After being processed, the buffers * are returned by adding the to the given queue. - * - * @param channelID The id describing the path of the file that the channel accessed. + * + * @param channelID The id describing the path of the file that the channel accessed. * @param requestQueue The queue that this channel hands its IO requests to. - * @param callback The callback to be invoked when a request is done. + * @param callback The callback to be invoked when a request is done. * @param writeEnabled Flag describing whether the channel should be opened in read/write mode, rather * than in read-only mode. * @throws IOException Thrown, if the channel could no be opened. */ - protected AsynchronousFileIOChannel(FileIOChannel.ID channelID, RequestQueue requestQueue, - RequestDoneCallback callback, boolean writeEnabled) throws IOException - { + protected AsynchronousFileIOChannel(FileIOChannel.ID channelID, RequestQueue requestQueue, + RequestDoneCallback callback, boolean writeEnabled) throws IOException { super(channelID, writeEnabled); this.requestQueue = checkNotNull(requestQueue); this.resultHandler = checkNotNull(callback); } - + // -------------------------------------------------------------------------------------------- - + @Override public boolean isClosed() { return this.closed; } - + /** * Closes the reader and waits until all pending asynchronous requests are * handled. Even if an exception interrupts the closing, the underlying FileChannel is closed. - * + * * @throws IOException Thrown, if an I/O exception occurred while waiting for the buffers, or if * the closing was interrupted. */ + @Override public void close() throws IOException { // atomically set the close flag synchronized (this.closeLock) { @@ -101,7 +111,7 @@ public void close() throws IOException { return; } this.closed = true; - + try { // wait until as many buffers have been returned as were written // only then is everything guaranteed to be consistent. @@ -117,8 +127,7 @@ public void close() throws IOException { throw new IOException("Closing of asynchronous file channel was interrupted."); } } - } - finally { + } finally { // close the file if (this.fileChannel.isOpen()) { this.fileChannel.close(); @@ -126,16 +135,17 @@ public void close() throws IOException { } } } - + /** * This method waits for all pending asynchronous requests to return. When the * last request has returned, the channel is closed and deleted. *

* Even if an exception interrupts the closing, such that not all request are handled, * the underlying FileChannel is closed and deleted. - * + * * @throws IOException Thrown, if an I/O exception occurred while waiting for the buffers, or if the closing was interrupted. */ + @Override public void closeAndDelete() throws IOException { try { close(); @@ -144,11 +154,11 @@ public void closeAndDelete() throws IOException { deleteChannel(); } } - + /** * Checks the exception state of this channel. The channel is erroneous, if one of its requests could not * be processed correctly. - * + * * @throws IOException Thrown, if the channel is erroneous. The thrown exception contains the original exception * that defined the erroneous state as its cause. */ @@ -157,15 +167,15 @@ public final void checkErroneous() throws IOException { throw this.exception; } } - + /** * Handles a processed Buffer. This method is invoked by the * asynchronous IO worker threads upon completion of the IO request with the * provided buffer and/or an exception that occurred while processing the request * for that buffer. - * + * * @param buffer The buffer to be processed. - * @param ex The exception that occurred in the I/O threads when processing the buffer's request. + * @param ex The exception that occurred in the I/O threads when processing the buffer's request. */ final protected void handleProcessedBuffer(T buffer, IOException ex) { if (buffer == null) { @@ -183,11 +193,19 @@ final protected void handleProcessedBuffer(T buffer, IOException ex) { } } finally { - // decrement the number of missing buffers. If we are currently closing, notify the waiters - synchronized (this.closeLock) { - final int num = this.requestsNotReturned.decrementAndGet(); - if (this.closed && num == 0) { - this.closeLock.notifyAll(); + // decrement the number of missing buffers. If we are currently closing, notify the + if (this.closed) { + synchronized (this.closeLock) { + if (this.requestsNotReturned.decrementAndGet() == 0) { + maybeNotifyListener(); + + this.closeLock.notifyAll(); + } + } + } + else { + if (this.requestsNotReturned.decrementAndGet() == 0) { + maybeNotifyListener(); } } } @@ -207,19 +225,79 @@ final protected void addRequest(R request) throws IOException { } this.requestQueue.add(request); } + + protected boolean subscribe(NotificationListener listener) throws IOException { + if (allRequestsProcessedListener == null) { + if (requestsNotReturned.get() > 0) { + synchronized (subscribeLock) { + allRequestsProcessedListener = listener; + + // There was a race with the processing of the last outstanding request + if (requestsNotReturned.get() == 0) { + allRequestsProcessedListener = null; + return false; + } + } + + return true; + } + + return false; + } + + throw new ResultSubpartitionView.AlreadySubscribedException(); + } + + private void maybeNotifyListener() { + synchronized (subscribeLock) { + if (allRequestsProcessedListener != null) { + NotificationListener listener = allRequestsProcessedListener; + allRequestsProcessedListener = null; + listener.onNotification(); + } + } + } } //-------------------------------------------------------------------------------------------- +/** + * Request that seeks the underlying file channel to the given position. + */ +final class SeekRequest implements ReadRequest, WriteRequest { + + private final AsynchronousFileIOChannel channel; + private final long position; + + protected SeekRequest(AsynchronousFileIOChannel channel, long position) { + this.channel = channel; + this.position = position; + } + + @Override + public void read() throws IOException { + channel.fileChannel.position(position); + } + + @Override + public void write() throws IOException { + channel.fileChannel.position(position); + } + + @Override + public void requestDone(IOException error) { + } +} + /** * Read request that reads an entire memory segment from a block reader. */ final class SegmentReadRequest implements ReadRequest { - + private final AsynchronousFileIOChannel channel; - + private final MemorySegment segment; - + protected SegmentReadRequest(AsynchronousFileIOChannel targetChannel, MemorySegment segment) { this.channel = targetChannel; this.segment = segment; @@ -251,11 +329,11 @@ public void requestDone(IOException ioex) { * Write request that writes an entire memory segment to the block writer. */ final class SegmentWriteRequest implements WriteRequest { - + private final AsynchronousFileIOChannel channel; - + private final MemorySegment segment; - + protected SegmentWriteRequest(AsynchronousFileIOChannel targetChannel, MemorySegment segment) { this.channel = targetChannel; this.segment = segment; @@ -277,30 +355,131 @@ public void requestDone(IOException ioex) { } } -/** - * Request that seeks the underlying file channel to the given position. - */ -final class SeekRequest implements ReadRequest, WriteRequest { +final class BufferWriteRequest implements WriteRequest { - private final AsynchronousFileIOChannel channel; - private final long position; + private final AsynchronousFileIOChannel channel; - protected SeekRequest(AsynchronousFileIOChannel channel, long position) { - this.channel = channel; - this.position = position; + private final Buffer buffer; + + protected BufferWriteRequest(AsynchronousFileIOChannel targetChannel, Buffer buffer) { + this.channel = targetChannel; + this.buffer = buffer; } @Override - public void requestDone(IOException ioex) { + public void write() throws IOException { + final ByteBuffer header = ByteBuffer.allocateDirect(8); + + header.putInt(buffer.isBuffer() ? 1 : 0); + header.putInt(buffer.getSize()); + header.flip(); + + channel.fileChannel.write(header); + channel.fileChannel.write(buffer.getNioBuffer()); + } + + @Override + public void requestDone(IOException error) { + channel.handleProcessedBuffer(buffer, error); + } +} + +final class BufferReadRequest implements ReadRequest { + + private final AsynchronousFileIOChannel channel; + + private final Buffer buffer; + + private final AtomicBoolean isConsumed; + + protected BufferReadRequest(AsynchronousFileIOChannel targetChannel, Buffer buffer, AtomicBoolean isConsumed) { + this.channel = targetChannel; + this.buffer = buffer; + this.isConsumed = isConsumed; } @Override public void read() throws IOException { - this.channel.fileChannel.position(position); + + final FileChannel fileChannel = channel.fileChannel; + + if (fileChannel.size() - fileChannel.position() > 0) { + final ByteBuffer header = ByteBuffer.allocateDirect(8); + + fileChannel.read(header); + header.flip(); + + final boolean isBuffer = header.getInt() == 1; + final int size = header.getInt(); + + if (size > buffer.getMemorySegment().size()) { + throw new IllegalStateException("Buffer is too small for data: " + buffer.getMemorySegment().size() + " bytes available, but " + size + " needed. This is most likely due to an serialized event, which is larger than the buffer size."); + } + + buffer.setSize(size); + + fileChannel.read(buffer.getNioBuffer()); + + if (!isBuffer) { + buffer.tagAsEvent(); + } + + isConsumed.set(fileChannel.size() - fileChannel.position() == 0); + } + else { + isConsumed.set(true); + } } @Override - public void write() throws IOException { - this.channel.fileChannel.position(position); + public void requestDone(IOException error) { + channel.handleProcessedBuffer(buffer, error); + } +} + +final class FileSegmentReadRequest implements ReadRequest { + + private final AsynchronousFileIOChannel channel; + + private final AtomicBoolean isConsumed; + + private FileSegment fileSegment; + + protected FileSegmentReadRequest(AsynchronousFileIOChannel targetChannel, AtomicBoolean isConsumed) { + this.channel = targetChannel; + this.isConsumed = isConsumed; + } + + @Override + public void read() throws IOException { + + final FileChannel fileChannel = channel.fileChannel; + + if (fileChannel.size() - fileChannel.position() > 0) { + final ByteBuffer header = ByteBuffer.allocateDirect(8); + + fileChannel.read(header); + header.flip(); + + final long position = fileChannel.position(); + + final boolean isBuffer = header.getInt() == 1; + final int length = header.getInt(); + + fileSegment = new FileSegment(fileChannel, position, length, isBuffer); + + // Skip the binary data + fileChannel.position(position + length); + + isConsumed.set(fileChannel.size() - fileChannel.position() == 0); + } + else { + isConsumed.set(true); + } + } + + @Override + public void requestDone(IOException error) { + channel.handleProcessedBuffer(fileSegment, error); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelReader.java index 8f7f218dcbb50..dbbdfb8a45c4a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelReader.java @@ -21,15 +21,13 @@ import java.io.IOException; import java.util.concurrent.LinkedBlockingQueue; -import org.apache.flink.core.memory.MemorySegment; - /** * A reader that reads data in blocks from a file channel. The reader reads the blocks into a * {@link org.apache.flink.core.memory.MemorySegment}. To support asynchronous implementations, * the read method does not immediately return the full memory segment, but rather adds it to * a blocking queue of finished read operations. */ -public interface BlockChannelReader extends FileIOChannel { +public interface BlockChannelReader extends FileIOChannel { /** * Issues a read request, which will fill the given segment with the next block in the @@ -39,33 +37,27 @@ public interface BlockChannelReader extends FileIOChannel { * @param segment The segment to read the block into. * @throws IOException Thrown, when the reader encounters an I/O error. */ - void readBlock(MemorySegment segment) throws IOException; + void readBlock(T segment) throws IOException; + + void seekToPosition(long position) throws IOException; /** * Gets the next memory segment that has been filled with data by the reader. This method blocks until * such a segment is available, or until an error occurs in the reader, or the reader is closed. *

* WARNING: If this method is invoked without any segment ever returning (for example, because the - * {@link #readBlock(MemorySegment)} method has not been invoked appropriately), the method may block + * {@link BlockChannelReader#readBlock(T)} method has not been invoked appropriately), the method may block * forever. * * @return The next memory segment from the reader's return queue. * @throws IOException Thrown, if an I/O error occurs in the reader while waiting for the request to return. */ - public MemorySegment getNextReturnedSegment() throws IOException; + public T getNextReturnedBlock() throws IOException; /** * Gets the queue in which the full memory segments are queued after the read is complete. * * @return The queue with the full memory segments. */ - LinkedBlockingQueue getReturnQueue(); - - /** - * Seeks the underlying file channel to the given position. - * - * @param position The position to seek to. - */ - void seekToPosition(long position) throws IOException; + LinkedBlockingQueue getReturnQueue(); } - \ No newline at end of file diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriter.java index 25c74e4125060..0dd0907c1c1de 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriter.java @@ -21,15 +21,13 @@ import java.io.IOException; import java.util.concurrent.LinkedBlockingQueue; -import org.apache.flink.core.memory.MemorySegment; - /** * A writer that writes data in blocks to a file channel. The writer receives the data blocks in the form of * {@link org.apache.flink.core.memory.MemorySegment}, which it writes entirely to the channel, * regardless of how space in the segment is used. The writing may be realized synchronously, or asynchronously, * depending on the implementation. */ -public interface BlockChannelWriter extends BlockChannelWriterWithCallback { +public interface BlockChannelWriter extends BlockChannelWriterWithCallback { /** * Gets the next memory segment that has been written and is available again. @@ -37,13 +35,13 @@ public interface BlockChannelWriter extends BlockChannelWriterWithCallback { * writer is closed. *

* NOTE: If this method is invoked without any segment ever returning (for example, because the - * {@link #writeBlock(MemorySegment)} method has not been invoked accordingly), the method may block + * {@link BlockChannelWriterWithCallback#writeBlock(T)} method has not been invoked accordingly), the method may block * forever. * * @return The next memory segment from the writers's return queue. * @throws IOException Thrown, if an I/O error occurs in the writer while waiting for the request to return. */ - MemorySegment getNextReturnedSegment() throws IOException; + T getNextReturnedBlock() throws IOException; /** * Gets the queue in which the memory segments are queued after the asynchronous write @@ -51,5 +49,5 @@ public interface BlockChannelWriter extends BlockChannelWriterWithCallback { * * @return The queue with the written memory segments. */ - LinkedBlockingQueue getReturnQueue(); + LinkedBlockingQueue getReturnQueue(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriterWithCallback.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriterWithCallback.java index 57bc7e0d9b036..f7618e493993c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriterWithCallback.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BlockChannelWriterWithCallback.java @@ -20,16 +20,14 @@ import java.io.IOException; -import org.apache.flink.core.memory.MemorySegment; +public interface BlockChannelWriterWithCallback extends FileIOChannel { -public interface BlockChannelWriterWithCallback extends FileIOChannel { - /** - * Writes the given memory segment. The request may be executed synchronously, or asynchronously, depending + * Writes the given block. The request may be executed synchronously, or asynchronously, depending * on the implementation. - * - * @param segment The segment to be written. + * + * @param block The segment to be written. * @throws IOException Thrown, when the writer encounters an I/O error. */ - void writeBlock(MemorySegment segment) throws IOException; + void writeBlock(T block) throws IOException; } \ No newline at end of file diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileReader.java new file mode 100644 index 0000000000000..eb289fe106dea --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileReader.java @@ -0,0 +1,33 @@ +/* + * 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.disk.iomanager; + +import org.apache.flink.runtime.io.network.buffer.Buffer; + +import java.io.IOException; + +public interface BufferFileReader extends FileIOChannel { + + void readInto(Buffer buffer) throws IOException; + + void seekToPosition(long position) throws IOException; + + boolean isConsumed(); + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileSegmentReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileSegmentReader.java new file mode 100644 index 0000000000000..f19a02da59eab --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileSegmentReader.java @@ -0,0 +1,31 @@ +/* + * 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.disk.iomanager; + +import java.io.IOException; + +public interface BufferFileSegmentReader extends FileIOChannel { + + void read() throws IOException; + + void seekTo(long position) throws IOException; + + boolean isConsumed(); + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriter.java new file mode 100644 index 0000000000000..1e06d71195765 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriter.java @@ -0,0 +1,32 @@ +/* + * 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.disk.iomanager; + +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.util.event.NotificationListener; + +import java.io.IOException; + +public interface BufferFileWriter extends BlockChannelWriterWithCallback { + + int getNumberOfOutstandingRequests(); + + boolean subscribe(NotificationListener listener) throws IOException; + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java index d85ec8212d41f..b9190345eb5b1 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java @@ -35,7 +35,7 @@ */ public class ChannelReaderInputView extends AbstractPagedInputView { - protected final BlockChannelReader reader; // the block reader that reads memory segments + protected final BlockChannelReader reader; // the block reader that reads memory segments protected int numRequestsRemaining; // the number of block requests remaining @@ -63,7 +63,7 @@ public class ChannelReaderInputView extends AbstractPagedInputView { * @throws IOException Thrown, if the read requests for the first blocks fail to be * served by the reader. */ - public ChannelReaderInputView(BlockChannelReader reader, List memory, boolean waitForFirstBlock) + public ChannelReaderInputView(BlockChannelReader reader, List memory, boolean waitForFirstBlock) throws IOException { this(reader, memory, -1, waitForFirstBlock); @@ -89,7 +89,7 @@ public ChannelReaderInputView(BlockChannelReader reader, List mem * @throws IOException Thrown, if the read requests for the first blocks fail to be * served by the reader. */ - public ChannelReaderInputView(BlockChannelReader reader, List memory, + public ChannelReaderInputView(BlockChannelReader reader, List memory, int numBlocks, boolean waitForFirstBlock) throws IOException { @@ -117,7 +117,7 @@ public ChannelReaderInputView(BlockChannelReader reader, List mem * * @throws IOException */ - ChannelReaderInputView(BlockChannelReader reader, List memory, + ChannelReaderInputView(BlockChannelReader reader, List memory, int numBlocks, int headerLen, boolean waitForFirstBlock) throws IOException { @@ -225,7 +225,7 @@ protected MemorySegment nextSegment(MemorySegment current) throws IOException { } // get the next segment - final MemorySegment seg = this.reader.getNextReturnedSegment(); + final MemorySegment seg = this.reader.getNextReturnedBlock(); // check the header if (seg.getShort(0) != ChannelWriterOutputView.HEADER_MAGIC_NUMBER) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java index 9824d34fad6b1..089e10a78228e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java @@ -61,7 +61,7 @@ public final class ChannelWriterOutputView extends AbstractPagedOutputView { // -------------------------------------------------------------------------------------------- - private final BlockChannelWriter writer; // the writer to the channel + private final BlockChannelWriter writer; // the writer to the channel private long bytesBeforeSegment; // the number of bytes written before the current memory segment @@ -81,7 +81,7 @@ public final class ChannelWriterOutputView extends AbstractPagedOutputView { * @param memory The memory used to buffer data, or null, to utilize solely the return queue. * @param segmentSize The size of the memory segments. */ - public ChannelWriterOutputView(BlockChannelWriter writer, List memory, int segmentSize) { + public ChannelWriterOutputView(BlockChannelWriter writer, List memory, int segmentSize) { super(segmentSize, HEADER_LENGTH); if (writer == null) { @@ -123,7 +123,7 @@ public ChannelWriterOutputView(BlockChannelWriter writer, List me * @param writer The writer to write to. * @param segmentSize The size of the memory segments. */ - public ChannelWriterOutputView(BlockChannelWriter writer, int segmentSize) + public ChannelWriterOutputView(BlockChannelWriter writer, int segmentSize) { this(writer, null, segmentSize); } @@ -203,7 +203,7 @@ protected final MemorySegment nextSegment(MemorySegment current, int posInSegmen writeSegment(current, posInSegment, false); } - final MemorySegment next = this.writer.getNextReturnedSegment(); + final MemorySegment next = this.writer.getNextReturnedBlock(); this.blockCount++; return next; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/FileSegment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/FileSegment.java new file mode 100644 index 0000000000000..7c3a83ed44952 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/FileSegment.java @@ -0,0 +1,52 @@ +/* + * 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.disk.iomanager; + +import java.nio.channels.FileChannel; + +public class FileSegment { + + private final FileChannel fileChannel; + private final long position; + private final int length; + private final boolean isBuffer; + + public FileSegment(FileChannel fileChannel, long position, int length, boolean isBuffer) { + this.fileChannel = fileChannel; + this.position = position; + this.length = length; + this.isBuffer = isBuffer; + } + + public FileChannel getFileChannel() { + return fileChannel; + } + + public long getPosition() { + return position; + } + + public int getLength() { + return length; + } + + public boolean isBuffer() { + return isBuffer; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/HeaderlessChannelReaderInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/HeaderlessChannelReaderInputView.java index cdad3fb42ee9b..63e86c9d97e1b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/HeaderlessChannelReaderInputView.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/HeaderlessChannelReaderInputView.java @@ -60,7 +60,7 @@ public class HeaderlessChannelReaderInputView extends ChannelReaderInputView * @throws IOException Thrown, if the read requests for the first blocks fail to be * served by the reader. */ - public HeaderlessChannelReaderInputView(BlockChannelReader reader, List memory, int numBlocks, + public HeaderlessChannelReaderInputView(BlockChannelReader reader, List memory, int numBlocks, int numBytesInLastBlock, boolean waitForFirstBlock) throws IOException { @@ -87,7 +87,7 @@ protected MemorySegment nextSegment(MemorySegment current) throws IOException { // get the next segment this.numBlocksRemaining--; - return this.reader.getNextReturnedSegment(); + return this.reader.getNextReturnedBlock(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java index 6cf19f35c7127..9e1f41c9ae963 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.io.disk.iomanager; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.network.buffer.Buffer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -123,7 +124,7 @@ public void deleteChannel(FileIOChannel.ID channel) throws IOException { * @return A block channel writer that writes to the given channel. * @throws IOException Thrown, if the channel for the writer could not be opened. */ - public BlockChannelWriter createBlockChannelWriter(FileIOChannel.ID channelID) throws IOException { + public BlockChannelWriter createBlockChannelWriter(FileIOChannel.ID channelID) throws IOException { return createBlockChannelWriter(channelID, new LinkedBlockingQueue()); } @@ -136,7 +137,7 @@ public BlockChannelWriter createBlockChannelWriter(FileIOChannel.ID channelID) t * @return A block channel writer that writes to the given channel. * @throws IOException Thrown, if the channel for the writer could not be opened. */ - public abstract BlockChannelWriter createBlockChannelWriter(FileIOChannel.ID channelID, + public abstract BlockChannelWriter createBlockChannelWriter(FileIOChannel.ID channelID, LinkedBlockingQueue returnQueue) throws IOException; /** @@ -149,7 +150,7 @@ public abstract BlockChannelWriter createBlockChannelWriter(FileIOChannel.ID cha * @return A block channel writer that writes to the given channel. * @throws IOException Thrown, if the channel for the writer could not be opened. */ - public abstract BlockChannelWriterWithCallback createBlockChannelWriter(FileIOChannel.ID channelID, RequestDoneCallback callback) throws IOException; + public abstract BlockChannelWriterWithCallback createBlockChannelWriter(FileIOChannel.ID channelID, RequestDoneCallback callback) throws IOException; /** * Creates a block channel reader that reads blocks from the given channel. The reader pushed @@ -160,7 +161,7 @@ public abstract BlockChannelWriter createBlockChannelWriter(FileIOChannel.ID cha * @return A block channel reader that reads from the given channel. * @throws IOException Thrown, if the channel for the reader could not be opened. */ - public BlockChannelReader createBlockChannelReader(FileIOChannel.ID channelID) throws IOException { + public BlockChannelReader createBlockChannelReader(FileIOChannel.ID channelID) throws IOException { return createBlockChannelReader(channelID, new LinkedBlockingQueue()); } @@ -173,9 +174,15 @@ public BlockChannelReader createBlockChannelReader(FileIOChannel.ID channelID) t * @return A block channel reader that reads from the given channel. * @throws IOException Thrown, if the channel for the reader could not be opened. */ - public abstract BlockChannelReader createBlockChannelReader(FileIOChannel.ID channelID, + public abstract BlockChannelReader createBlockChannelReader(FileIOChannel.ID channelID, LinkedBlockingQueue returnQueue) throws IOException; + public abstract BufferFileWriter createBufferFileWriter(FileIOChannel.ID channelID) throws IOException; + + public abstract BufferFileReader createBufferFileReader(FileIOChannel.ID channelID, RequestDoneCallback callback) throws IOException; + + public abstract BufferFileSegmentReader createBufferFileSegmentReader(FileIOChannel.ID channelID, RequestDoneCallback callback) throws IOException; + /** * Creates a block channel reader that reads all blocks from the given channel directly in one bulk. * The reader draws segments to read the blocks into from a supplied list, which must contain as many diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsync.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsync.java index 6489396ebd69a..9ecd05fe720af 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsync.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsync.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.io.disk.iomanager; import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.util.EnvironmentInformation; import java.io.IOException; @@ -178,7 +179,7 @@ public void uncaughtException(Thread t, Throwable e) { // ------------------------------------------------------------------------ @Override - public BlockChannelWriter createBlockChannelWriter(FileIOChannel.ID channelID, + public BlockChannelWriter createBlockChannelWriter(FileIOChannel.ID channelID, LinkedBlockingQueue returnQueue) throws IOException { checkState(!shutdown, "I/O-Manger is closed."); @@ -186,7 +187,7 @@ public BlockChannelWriter createBlockChannelWriter(FileIOChannel.ID channelID, } @Override - public BlockChannelWriterWithCallback createBlockChannelWriter(FileIOChannel.ID channelID, RequestDoneCallback callback) throws IOException { + public BlockChannelWriterWithCallback createBlockChannelWriter(FileIOChannel.ID channelID, RequestDoneCallback callback) throws IOException { checkState(!shutdown, "I/O-Manger is closed."); return new AsynchronousBlockWriterWithCallback(channelID, this.writers[channelID.getThreadNum()].requestQueue, callback); } @@ -202,13 +203,32 @@ public BlockChannelWriterWithCallback createBlockChannelWriter(FileIOChannel.ID * @throws IOException Thrown, if the channel for the reader could not be opened. */ @Override - public BlockChannelReader createBlockChannelReader(FileIOChannel.ID channelID, + public BlockChannelReader createBlockChannelReader(FileIOChannel.ID channelID, LinkedBlockingQueue returnQueue) throws IOException { checkState(!shutdown, "I/O-Manger is closed."); return new AsynchronousBlockReader(channelID, this.readers[channelID.getThreadNum()].requestQueue, returnQueue); } - + + @Override + public BufferFileWriter createBufferFileWriter(FileIOChannel.ID channelID) throws IOException { + checkState(!shutdown, "I/O-Manger is closed."); + + return new AsynchronousBufferFileWriter(channelID, writers[channelID.getThreadNum()].requestQueue); + } + + @Override + public BufferFileReader createBufferFileReader(FileIOChannel.ID channelID, RequestDoneCallback callback) throws IOException { + checkState(!shutdown, "I/O-Manger is closed."); + + return new AsynchronousBufferFileReader(channelID, readers[channelID.getThreadNum()].requestQueue, callback); + } + + @Override + public BufferFileSegmentReader createBufferFileSegmentReader(FileIOChannel.ID channelID, RequestDoneCallback callback) throws IOException { + return new AsynchronousBufferFileSegmentReader(channelID, readers[channelID.getThreadNum()].requestQueue, callback); + } + /** * Creates a block channel reader that reads all blocks from the given channel directly in one bulk. * The reader draws segments to read the blocks into from a supplied list, which must contain as many diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/QueuingCallback.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/QueuingCallback.java index 95f3dc7a62d70..a2e3e82a78a2e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/QueuingCallback.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/QueuingCallback.java @@ -21,26 +21,24 @@ import java.io.IOException; import java.util.concurrent.LinkedBlockingQueue; -import org.apache.flink.core.memory.MemorySegment; - /** * A {@link RequestDoneCallback} that adds the memory segments to a blocking queue. */ -public class QueuingCallback implements RequestDoneCallback { +public class QueuingCallback implements RequestDoneCallback { + + private final LinkedBlockingQueue queue; - private final LinkedBlockingQueue queue; - - public QueuingCallback(LinkedBlockingQueue queue) { + public QueuingCallback(LinkedBlockingQueue queue) { this.queue = queue; } @Override - public void requestSuccessful(MemorySegment buffer) { + public void requestSuccessful(T buffer) { queue.add(buffer); } @Override - public void requestFailed(MemorySegment buffer, IOException e) { + public void requestFailed(T buffer, IOException e) { // the I/O error is recorded in the writer already queue.add(buffer); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionManager.java index 4a5536bd2d384..14ca04355d38a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ConnectionManager.java @@ -18,8 +18,9 @@ package org.apache.flink.runtime.io.network; +import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.netty.PartitionRequestClient; -import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionProvider; +import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; import java.io.IOException; @@ -29,7 +30,7 @@ */ public interface ConnectionManager { - void start(IntermediateResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher) throws IOException; + void start(ResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher, BufferPool bufferPool) throws IOException; /** * Creates a {@link PartitionRequestClient} instance for the given {@link RemoteAddress}. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/LocalConnectionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/LocalConnectionManager.java index 894db350c8e6f..29a9907c58c08 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/LocalConnectionManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/LocalConnectionManager.java @@ -18,8 +18,9 @@ package org.apache.flink.runtime.io.network; +import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.netty.PartitionRequestClient; -import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionProvider; +import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; import java.io.IOException; @@ -30,7 +31,8 @@ public class LocalConnectionManager implements ConnectionManager { @Override - public void start(IntermediateResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher) throws IOException { + public void start(ResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher, BufferPool bufferPool) throws IOException { + } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java index aa6c64c5c3fa3..a41dad99e1b97 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java @@ -26,8 +26,8 @@ import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; import org.apache.flink.runtime.io.network.netty.NettyConfig; import org.apache.flink.runtime.io.network.netty.NettyConnectionManager; -import org.apache.flink.runtime.io.network.partition.IntermediateResultPartition; -import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionManager; +import org.apache.flink.runtime.io.network.partition.ResultPartition; +import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskManager; @@ -51,7 +51,7 @@ public class NetworkEnvironment { private final FiniteDuration jobManagerTimeout; - private final IntermediateResultPartitionManager partitionManager; + private final ResultPartitionManager partitionManager; private final TaskEventDispatcher taskEventDispatcher; @@ -59,6 +59,8 @@ public class NetworkEnvironment { private final ConnectionManager connectionManager; + private final BufferPool nettyBufferPool; + private boolean isShutdown; /** @@ -68,7 +70,7 @@ public NetworkEnvironment(ActorRef jobManager, FiniteDuration jobManagerTimeout, this.jobManager = checkNotNull(jobManager); this.jobManagerTimeout = checkNotNull(jobManagerTimeout); - this.partitionManager = new IntermediateResultPartitionManager(); + this.partitionManager = new ResultPartitionManager(); this.taskEventDispatcher = new TaskEventDispatcher(); // -------------------------------------------------------------------- @@ -88,8 +90,10 @@ public NetworkEnvironment(ActorRef jobManager, FiniteDuration jobManagerTimeout, connectionManager = nettyConfig.isDefined() ? new NettyConnectionManager(nettyConfig.get()) : new LocalConnectionManager(); + nettyBufferPool = networkBufferPool.createBufferPool(128, true); + try { - connectionManager.start(partitionManager, taskEventDispatcher); + connectionManager.start(partitionManager, taskEventDispatcher, nettyBufferPool); } catch (Throwable t) { throw new IOException("Failed to instantiate network connection manager: " + t.getMessage(), t); @@ -107,7 +111,7 @@ public FiniteDuration getJobManagerTimeout() { public void registerTask(Task task) throws IOException { final ExecutionAttemptID executionId = task.getExecutionId(); - final IntermediateResultPartition[] producedPartitions = task.getProducedPartitions(); + final ResultPartition[] producedPartitions = task.getProducedPartitions(); final BufferWriter[] writers = task.getWriters(); if (writers.length != producedPartitions.length) { @@ -115,20 +119,21 @@ public void registerTask(Task task) throws IOException { } for (int i = 0; i < producedPartitions.length; i++) { - final IntermediateResultPartition partition = producedPartitions[i]; + final ResultPartition partition = producedPartitions[i]; final BufferWriter writer = writers[i]; // Buffer pool for the partition BufferPool bufferPool = null; try { - bufferPool = networkBufferPool.createBufferPool(partition.getNumberOfQueues(), false); - partition.setBufferPool(bufferPool); + bufferPool = networkBufferPool.createBufferPool(partition.getNumberOfSubpartitions(), false); + partition.registerBufferPool(bufferPool); + partitionManager.registerIntermediateResultPartition(partition); } catch (Throwable t) { if (bufferPool != null) { - bufferPool.destroy(); + bufferPool.lazyDestroy(); } if (t instanceof IOException) { @@ -155,7 +160,7 @@ public void registerTask(Task task) throws IOException { } catch (Throwable t) { if (bufferPool != null) { - bufferPool.destroy(); + bufferPool.lazyDestroy(); } if (t instanceof IOException) { @@ -176,7 +181,7 @@ public void unregisterTask(Task task) { final ExecutionAttemptID executionId = task.getExecutionId(); if (task.isCanceledOrFailed()) { - partitionManager.failIntermediateResultPartitions(executionId); + partitionManager.releasePartitionsProducedBy(executionId); } taskEventDispatcher.unregisterWriters(executionId); @@ -197,7 +202,7 @@ public void unregisterTask(Task task) { } } - public IntermediateResultPartitionManager getPartitionManager() { + public ResultPartitionManager getPartitionManager() { return partitionManager; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java index 7a529b91f8a5b..5e9838ad8a24e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/TaskEventDispatcher.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.event.task.TaskEvent; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.network.api.writer.BufferWriter; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; import org.apache.flink.runtime.util.event.EventListener; import java.util.ArrayList; @@ -39,9 +39,9 @@ */ public class TaskEventDispatcher { - Table registeredWriters = HashBasedTable.create(); + Table registeredWriters = HashBasedTable.create(); - public void registerWriterForIncomingTaskEvents(ExecutionAttemptID executionId, IntermediateResultPartitionID partitionId, BufferWriter listener) { + public void registerWriterForIncomingTaskEvents(ExecutionAttemptID executionId, ResultPartitionID partitionId, BufferWriter listener) { synchronized (registeredWriters) { if (registeredWriters.put(executionId, partitionId, listener) != null) { throw new IllegalStateException("Event dispatcher already contains buffer writer."); @@ -51,13 +51,13 @@ public void registerWriterForIncomingTaskEvents(ExecutionAttemptID executionId, public void unregisterWriters(ExecutionAttemptID executionId) { synchronized (registeredWriters) { - List writersToUnregister = new ArrayList(); + List writersToUnregister = new ArrayList(); - for (IntermediateResultPartitionID partitionId : registeredWriters.row(executionId).keySet()) { + for (ResultPartitionID partitionId : registeredWriters.row(executionId).keySet()) { writersToUnregister.add(partitionId); } - for(IntermediateResultPartitionID partitionId : writersToUnregister) { + for(ResultPartitionID partitionId : writersToUnregister) { registeredWriters.remove(executionId, partitionId); } } @@ -69,7 +69,7 @@ public void unregisterWriters(ExecutionAttemptID executionId) { * This method is either called from a local input channel or the network * I/O thread on behalf of a remote input channel. */ - public boolean publish(ExecutionAttemptID executionId, IntermediateResultPartitionID partitionId, TaskEvent event) { + public boolean publish(ExecutionAttemptID executionId, ResultPartitionID partitionId, TaskEvent event) { EventListener listener = registeredWriters.get(executionId, partitionId); if (listener != null) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java index 91784f6771f33..3900d5161645f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/reader/BufferReader.java @@ -36,13 +36,13 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.BufferProvider; -import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionProvider; +import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel; import org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel; import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; import org.apache.flink.runtime.io.network.partition.consumer.UnknownInputChannel; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; import org.apache.flink.runtime.util.event.EventListener; import org.apache.flink.runtime.util.event.EventNotificationHandler; import org.slf4j.Logger; @@ -77,7 +77,7 @@ public final class BufferReader implements BufferReaderBase { private final int queueToRequest; - private final Map inputChannels; + private final Map inputChannels; private BufferPool bufferPool; @@ -137,7 +137,7 @@ public String getTaskNameWithSubtasks() { return environment.getTaskNameWithSubtasks(); } - public IntermediateResultPartitionProvider getIntermediateResultPartitionProvider() { + public ResultPartitionProvider getIntermediateResultPartitionProvider() { return networkEnvironment.getPartitionManager(); } @@ -168,7 +168,7 @@ public BufferProvider getBufferProvider() { return bufferPool; } - public void setInputChannel(IntermediateResultPartitionID partitionId, InputChannel inputChannel) { + public void setInputChannel(ResultPartitionID partitionId, InputChannel inputChannel) { synchronized (requestLock) { inputChannels.put(checkNotNull(partitionId), checkNotNull(inputChannel)); } @@ -181,7 +181,7 @@ public void updateInputChannel(PartitionInfo partitionInfo) throws IOException { return; } - final IntermediateResultPartitionID partitionId = partitionInfo.getPartitionId(); + final ResultPartitionID partitionId = partitionInfo.getPartitionId(); InputChannel current = inputChannels.get(partitionId); @@ -346,7 +346,7 @@ public void releaseAllResources() throws IOException { // The buffer pool can actually be destroyed immediately after the // reader received all of the data from the input channels. if (bufferPool != null) { - bufferPool.destroy(); + bufferPool.lazyDestroy(); } } finally { @@ -457,7 +457,7 @@ public static BufferReader create(RuntimeEnvironment runtimeEnvironment, Network for (PartitionInfo partition : partitions) { final ExecutionAttemptID producerExecutionId = partition.getProducerExecutionId(); - final IntermediateResultPartitionID partitionId = partition.getPartitionId(); + final ResultPartitionID partitionId = partition.getPartitionId(); final PartitionLocation producerLocation = partition.getProducerLocation(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BufferWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BufferWriter.java index 6cb1831f40921..f644583b18ad7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BufferWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/BufferWriter.java @@ -24,8 +24,8 @@ import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferProvider; -import org.apache.flink.runtime.io.network.partition.IntermediateResultPartition; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartition; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; import org.apache.flink.runtime.util.event.EventListener; import org.apache.flink.runtime.util.event.EventNotificationHandler; @@ -42,11 +42,11 @@ */ public final class BufferWriter implements EventListener { - private final IntermediateResultPartition partition; + private final ResultPartition partition; private final EventNotificationHandler taskEventHandler = new EventNotificationHandler(); - public BufferWriter(IntermediateResultPartition partition) { + public BufferWriter(ResultPartition partition) { this.partition = partition; } @@ -54,7 +54,7 @@ public BufferWriter(IntermediateResultPartition partition) { // Attributes // ------------------------------------------------------------------------ - public IntermediateResultPartitionID getPartitionId() { + public ResultPartitionID getPartitionId() { return partition.getPartitionId(); } @@ -63,7 +63,7 @@ public BufferProvider getBufferProvider() { } public int getNumberOfOutputChannels() { - return partition.getNumberOfQueues(); + return partition.getNumberOfSubpartitions(); } // ------------------------------------------------------------------------ @@ -79,14 +79,14 @@ public void writeEvent(AbstractEvent event, int targetChannel) throws IOExceptio } public void writeEventToAllChannels(AbstractEvent event) throws IOException { - for (int i = 0; i < partition.getNumberOfQueues(); i++) { + for (int i = 0; i < partition.getNumberOfSubpartitions(); i++) { Buffer buffer = EventSerializer.toBuffer(event); partition.add(buffer, i); } } public void writeEndOfSuperstep() throws IOException { - for (int i = 0; i < partition.getNumberOfQueues(); i++) { + for (int i = 0; i < partition.getNumberOfSubpartitions(); i++) { Buffer buffer = EventSerializer.toBuffer(EndOfSuperstepEvent.INSTANCE); partition.add(buffer, i); } @@ -96,10 +96,6 @@ public void finish() throws IOException, InterruptedException { partition.finish(); } - public boolean isFinished() { - return partition.isFinished(); - } - // ------------------------------------------------------------------------ // Event handling // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java index ae87aff439e5c..2f57a984a24f7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/RecordWriter.java @@ -73,10 +73,6 @@ public RecordWriter(BufferWriter writer, ChannelSelector channelSelector) { } } - public boolean isFinished() { - return writer.isFinished(); - } - public void emit(T record) throws IOException, InterruptedException { for (int targetChannel : channelSelector.selectChannels(record, numChannels)) { // serialize with corresponding serializer and send full buffer diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java index 24cd1065142e7..68a608d42adc0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java @@ -39,7 +39,7 @@ public class Buffer { /** The recycler for the backing {@link MemorySegment} */ private final BufferRecycler recycler; - private final boolean isBuffer; + private boolean isBuffer; /** The current number of references to this buffer */ private int referenceCount = 1; @@ -66,6 +66,14 @@ public boolean isBuffer() { return isBuffer; } + public void tagAsEvent() { + synchronized (recycleLock) { + ensureNotRecycled(); + } + + isBuffer = false; + } + public MemorySegment getMemorySegment() { synchronized (recycleLock) { ensureNotRecycled(); @@ -84,8 +92,6 @@ public ByteBuffer getNioBuffer() { public int getSize() { synchronized (recycleLock) { - ensureNotRecycled(); - return currentSize; } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPool.java index 15ac7a37099a4..4d34a3077297b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPool.java @@ -24,7 +24,7 @@ public interface BufferPool extends BufferProvider, BufferRecycler { void setBufferPoolOwner(BufferPoolOwner owner); - void destroy() throws IOException; + void lazyDestroy(); boolean isDestroyed(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPoolOwner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPoolOwner.java index 65c948a4c9326..66a6995600841 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPoolOwner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/BufferPoolOwner.java @@ -22,6 +22,6 @@ public interface BufferPoolOwner { - void recycleBuffers(int numBuffersToRecycle) throws IOException; + void releaseMemory(int numBuffersToRecycle) throws IOException; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java index 9e4b5a1d6500a..eb059a4025527 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPool.java @@ -124,7 +124,7 @@ public Buffer requestBufferBlocking() throws IOException, InterruptedException { return requestBuffer(true); } - private Buffer requestBuffer(boolean isBlocking) throws InterruptedException { + private Buffer requestBuffer(boolean isBlocking) throws InterruptedException, IOException { synchronized (availableMemorySegments) { if (isDestroyed) { return null; @@ -132,6 +132,8 @@ private Buffer requestBuffer(boolean isBlocking) throws InterruptedException { returnExcessMemorySegments(); + boolean askToRecycle = owner != null; + while (availableMemorySegments.isEmpty()) { if (numberOfRequestedMemorySegments < currentPoolSize) { final MemorySegment segment = networkBufferPool.requestMemorySegment(); @@ -144,6 +146,10 @@ private Buffer requestBuffer(boolean isBlocking) throws InterruptedException { } } + if (askToRecycle) { + owner.releaseMemory(1); + } + if (isBlocking) { availableMemorySegments.wait(2000); } @@ -186,7 +192,7 @@ public void recycle(MemorySegment segment) { * Destroy is called after the produce or consume phase of a task finishes. */ @Override - public void destroy() throws IOException { + public void lazyDestroy() { synchronized (availableMemorySegments) { if (!isDestroyed) { MemorySegment segment; @@ -230,7 +236,7 @@ public void setNumBuffers(int numBuffers) throws IOException { // If there is a registered owner and we have still requested more buffers than our // size, trigger a recycle via the owner. if (owner != null && numberOfRequestedMemorySegments > currentPoolSize) { - owner.recycleBuffers(numberOfRequestedMemorySegments - numBuffers); + owner.releaseMemory(numberOfRequestedMemorySegments - numBuffers); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java index 194448901127e..6534ee7b38bae 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java @@ -176,14 +176,18 @@ public BufferPool createBufferPool(int numRequiredBuffers, boolean isFixedSize) } @Override - public void destroyBufferPool(BufferPool bufferPool) throws IOException { + public void destroyBufferPool(BufferPool bufferPool) { synchronized (factoryLock) { if (allBufferPools.remove(bufferPool)) { managedBufferPools.remove(bufferPool); numTotalRequiredBuffers -= bufferPool.getNumberOfRequiredMemorySegments(); - redistributeBuffers(); + try { + redistributeBuffers(); + } catch (IOException e) { + throw new RuntimeException(e); + } } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManager.java index bbb303bdd97ff..1611644575a34 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManager.java @@ -21,7 +21,8 @@ import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.RemoteAddress; import org.apache.flink.runtime.io.network.TaskEventDispatcher; -import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionProvider; +import org.apache.flink.runtime.io.network.buffer.BufferPool; +import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; import java.io.IOException; @@ -41,13 +42,12 @@ public NettyConnectionManager(NettyConfig nettyConfig) { } @Override - public void start(IntermediateResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher) throws IOException { - PartitionRequestProtocol partitionRequestProtocol = new PartitionRequestProtocol(partitionProvider, taskEventDispatcher); + public void start(ResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher, BufferPool bufferPool) throws IOException { + PartitionRequestProtocol partitionRequestProtocol = new PartitionRequestProtocol(partitionProvider, taskEventDispatcher, bufferPool); client.init(partitionRequestProtocol); server.init(partitionRequestProtocol); } - @Override public PartitionRequestClient createPartitionRequestClient(RemoteAddress remoteAddress) throws IOException { return partitionRequestClientFactory.createPartitionRequestClient(remoteAddress); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java index 39a03ac406b9b..e9b44fb5278e7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyMessage.java @@ -35,7 +35,7 @@ import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.StringUtils; @@ -334,7 +334,7 @@ static class PartitionRequest extends NettyMessage { ExecutionAttemptID producerExecutionId; - IntermediateResultPartitionID partitionId; + ResultPartitionID partitionId; int queueIndex; @@ -343,7 +343,7 @@ static class PartitionRequest extends NettyMessage { public PartitionRequest() { } - PartitionRequest(ExecutionAttemptID producerExecutionId, IntermediateResultPartitionID partitionId, int queueIndex, InputChannelID receiverId) { + PartitionRequest(ExecutionAttemptID producerExecutionId, ResultPartitionID partitionId, int queueIndex, InputChannelID receiverId) { this.producerExecutionId = producerExecutionId; this.partitionId = partitionId; this.queueIndex = queueIndex; @@ -376,7 +376,7 @@ ByteBuf write(ByteBufAllocator allocator) throws IOException { @Override public void readFrom(ByteBuf buffer) { producerExecutionId = ExecutionAttemptID.fromByteBuf(buffer); - partitionId = IntermediateResultPartitionID.fromByteBuf(buffer); + partitionId = ResultPartitionID.fromByteBuf(buffer); queueIndex = buffer.readInt(); receiverId = InputChannelID.fromByteBuf(buffer); } @@ -392,12 +392,12 @@ static class TaskEventRequest extends NettyMessage { ExecutionAttemptID executionId; - IntermediateResultPartitionID partitionId; + ResultPartitionID partitionId; public TaskEventRequest() { } - TaskEventRequest(TaskEvent event, ExecutionAttemptID executionId, IntermediateResultPartitionID partitionId, InputChannelID receiverId) { + TaskEventRequest(TaskEvent event, ExecutionAttemptID executionId, ResultPartitionID partitionId, InputChannelID receiverId) { this.event = event; this.executionId = executionId; this.receiverId = receiverId; @@ -444,7 +444,7 @@ public void readFrom(ByteBuf buffer) { executionId = ExecutionAttemptID.fromByteBuf(buffer); receiverId = InputChannelID.fromByteBuf(buffer); - partitionId = IntermediateResultPartitionID.fromByteBuf(buffer); + partitionId = ResultPartitionID.fromByteBuf(buffer); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java index 25021e2fb4fb5..b1ac3c8f18773 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClient.java @@ -25,7 +25,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.network.RemoteAddress; import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; import org.apache.flink.runtime.util.AtomicDisposableReferenceCounter; import java.io.IOException; @@ -76,7 +76,7 @@ boolean incrementReferenceCounter() { * The request goes to the remote producer, for which this partition * request client instance has been created. */ - public void requestIntermediateResultPartition(ExecutionAttemptID producerExecutionId, final IntermediateResultPartitionID partitionId, final int requestedQueueIndex, final RemoteInputChannel inputChannel) throws IOException { + public void requestIntermediateResultPartition(ExecutionAttemptID producerExecutionId, final ResultPartitionID partitionId, final int requestedQueueIndex, final RemoteInputChannel inputChannel) throws IOException { partitionRequestHandler.addInputChannel(inputChannel); tcpChannel.writeAndFlush(new PartitionRequest(producerExecutionId, partitionId, requestedQueueIndex, inputChannel.getInputChannelId())) @@ -101,7 +101,7 @@ public void operationComplete(ChannelFuture future) throws Exception { * guaranteed to be the case when both the respective producer and * consumer task run pipelined. */ - public void sendTaskEvent(ExecutionAttemptID producerExecutionId, IntermediateResultPartitionID partitionId, TaskEvent event, final RemoteInputChannel inputChannel) throws IOException { + public void sendTaskEvent(ExecutionAttemptID producerExecutionId, ResultPartitionID partitionId, TaskEvent event, final RemoteInputChannel inputChannel) throws IOException { tcpChannel.writeAndFlush(new TaskEventRequest(event, producerExecutionId, partitionId, inputChannel.getInputChannelId())) .addListener( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java index dd8600429ce99..6177a7e0a5418 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientFactory.java @@ -31,7 +31,7 @@ /** * Factory for {@link PartitionRequestClient} instances. *

- * Instances of partition requests clients are shared among several {@link RemoteInputChannel} + * Instances of partition request clients are shared among several {@link RemoteInputChannel} * instances. */ class PartitionRequestClientFactory { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestProtocol.java index b8fbc10f6725e..4c9025b71c2c7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestProtocol.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestProtocol.java @@ -20,7 +20,8 @@ import io.netty.channel.ChannelPipeline; import org.apache.flink.runtime.io.network.TaskEventDispatcher; -import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionProvider; +import org.apache.flink.runtime.io.network.buffer.BufferPool; +import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; import static org.apache.flink.runtime.io.network.netty.NettyMessage.NettyMessageEncoder; import static org.apache.flink.runtime.io.network.netty.NettyMessage.NettyMessageEncoder.createFrameLengthDecoder; @@ -33,12 +34,14 @@ class PartitionRequestProtocol implements NettyProtocol { private final NettyMessage.NettyMessageDecoder messageDecoder = new NettyMessage.NettyMessageDecoder(); - private final IntermediateResultPartitionProvider partitionProvider; + private final ResultPartitionProvider partitionProvider; private final TaskEventDispatcher taskEventDispatcher; + private final BufferPool bufferPool; - PartitionRequestProtocol(IntermediateResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher) { + PartitionRequestProtocol(ResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher, BufferPool bufferPool) { this.partitionProvider = partitionProvider; this.taskEventDispatcher = taskEventDispatcher; + this.bufferPool = bufferPool; } // +-------------------------------------------------------------------+ @@ -80,7 +83,7 @@ public void setServerChannelPipeline(ChannelPipeline channelPipeline) { .addLast("Message encoder", messageEncoder) .addLast("Frame decoder", createFrameLengthDecoder()) .addLast("Client request decoder", messageDecoder) - .addLast("Server request handler", new PartitionRequestServerHandler(partitionProvider, taskEventDispatcher, queueOfPartitionQueues)) + .addLast("Server request handler", new PartitionRequestServerHandler(partitionProvider, taskEventDispatcher, queueOfPartitionQueues, bufferPool)) .addLast("Queue of queues", queueOfPartitionQueues); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java index d107d954f8a04..0c4dc63eb7a61 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestQueue.java @@ -24,7 +24,7 @@ import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator; +import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; import org.apache.flink.runtime.util.event.NotificationListener; import org.slf4j.Logger; @@ -50,9 +50,9 @@ class PartitionRequestQueue extends ChannelInboundHandlerAdapter { private final ChannelFutureListener writeListener = new WriteAndFlushNextMessageIfPossibleListener(); - private final Queue queue = new ArrayDeque(); + private final Queue queue = new ArrayDeque(); - private SequenceNumberingPartitionQueueIterator currentPartitionQueue; + private SequenceNumberingSubpartitionView currentPartitionQueue; private boolean fatalError; @@ -97,18 +97,18 @@ public void channelRegistered(final ChannelHandlerContext ctx) throws Exception super.channelRegistered(ctx); } - public void enqueue(IntermediateResultPartitionQueueIterator partitionQueue, InputChannelID receiverId) throws Exception { + public void enqueue(ResultSubpartitionView partitionQueue, InputChannelID receiverId) throws Exception { numEnqueueCalls.incrementAndGet(); - ctx.pipeline().fireUserEventTriggered(new SequenceNumberingPartitionQueueIterator(partitionQueue, receiverId)); + ctx.pipeline().fireUserEventTriggered(new SequenceNumberingSubpartitionView(partitionQueue, receiverId)); } @Override public void userEventTriggered(ChannelHandlerContext ctx, Object msg) throws Exception { - if (msg.getClass() == SequenceNumberingPartitionQueueIterator.class) { + if (msg.getClass() == SequenceNumberingSubpartitionView.class) { boolean triggerWrite = queue.isEmpty(); numTotalEnqueueOperations++; - queue.add((SequenceNumberingPartitionQueueIterator) msg); + queue.add((SequenceNumberingSubpartitionView) msg); if (triggerWrite) { writeAndFlushNextMessageIfPossible(ctx.channel()); @@ -164,13 +164,8 @@ else if (currentPartitionQueue.isConsumed()) { } } catch (Throwable t) { - try { - if (buffer != null) { - buffer.recycle(); - } - } - catch (Throwable ignored) { - // Make sure that this buffer is recycled in any case + if (buffer != null) { + buffer.recycle(); } throw new IOException(t.getMessage(), t); @@ -200,12 +195,12 @@ private void handleException(Channel channel, Throwable cause) throws IOExceptio private void releaseAllResources() throws IOException { if (currentPartitionQueue != null) { - currentPartitionQueue.discard(); + currentPartitionQueue.release(); currentPartitionQueue = null; } while ((currentPartitionQueue = queue.poll()) != null) { - currentPartitionQueue.discard(); + currentPartitionQueue.release(); } } @@ -232,15 +227,15 @@ else if (future.cause() != null) { * Simple wrapper for the partition queue iterator, which increments a * sequence number for each returned buffer and remembers the receiver ID. */ - private class SequenceNumberingPartitionQueueIterator implements IntermediateResultPartitionQueueIterator, NotificationListener { + private class SequenceNumberingSubpartitionView implements ResultSubpartitionView, NotificationListener { - private final IntermediateResultPartitionQueueIterator queueIterator; + private final ResultSubpartitionView queueIterator; private final InputChannelID receiverId; private int sequenceNumber = -1; - private SequenceNumberingPartitionQueueIterator(IntermediateResultPartitionQueueIterator queueIterator, InputChannelID receiverId) { + private SequenceNumberingSubpartitionView(ResultSubpartitionView queueIterator, InputChannelID receiverId) { this.queueIterator = checkNotNull(queueIterator); this.receiverId = checkNotNull(receiverId); } @@ -265,12 +260,12 @@ public Buffer getNextBuffer() throws IOException { } @Override - public void discard() throws IOException { - queueIterator.discard(); + public void release() throws IOException { + queueIterator.release(); } @Override - public boolean subscribe(NotificationListener ignored) throws AlreadySubscribedException { + public boolean subscribe(NotificationListener ignored) throws IOException { return queueIterator.subscribe(this); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java index 93f719241f75d..8e30eb536eb1e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestServerHandler.java @@ -22,10 +22,11 @@ import io.netty.channel.ChannelHandlerContext; import io.netty.channel.SimpleChannelInboundHandler; import org.apache.flink.runtime.io.network.TaskEventDispatcher; +import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.BufferProvider; -import org.apache.flink.runtime.io.network.partition.IntermediateResultPartitionProvider; +import org.apache.flink.runtime.io.network.partition.ResultPartitionProvider; import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; -import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator; +import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,17 +37,19 @@ class PartitionRequestServerHandler extends SimpleChannelInboundHandlerabsent()); + ResultSubpartitionView queueIterator = + partitionProvider.getSubpartition( + request.producerExecutionId, request.partitionId, request.queueIndex, + Optional.of(bufferPool)); if (queueIterator != null) { outboundQueue.enqueue(queueIterator, request.receiverId); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IllegalQueueIteratorRequestException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IllegalSubpartitionRequestException.java similarity index 79% rename from flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IllegalQueueIteratorRequestException.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IllegalSubpartitionRequestException.java index 80284c4a9f6eb..113012a03ca66 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IllegalQueueIteratorRequestException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IllegalSubpartitionRequestException.java @@ -16,18 +16,18 @@ * limitations under the License. */ -package org.apache.flink.runtime.io.network.partition.queue; +package org.apache.flink.runtime.io.network.partition; import java.io.IOException; -public class IllegalQueueIteratorRequestException extends IOException { +public class IllegalSubpartitionRequestException extends IOException { private static final long serialVersionUID = 8381253563445306324L; - public IllegalQueueIteratorRequestException() { + public IllegalSubpartitionRequestException() { } - public IllegalQueueIteratorRequestException(String message) { + public IllegalSubpartitionRequestException(String message) { super(message); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartition.java deleted file mode 100644 index 6acfbcefdcf93..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartition.java +++ /dev/null @@ -1,307 +0,0 @@ -/* - * 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; - -import com.google.common.base.Optional; -import org.apache.flink.runtime.akka.AkkaUtils; -import org.apache.flink.runtime.deployment.PartitionDeploymentDescriptor; -import org.apache.flink.runtime.execution.RuntimeEnvironment; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.executiongraph.ExecutionVertex; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.buffer.BufferPool; -import org.apache.flink.runtime.io.network.buffer.BufferPoolOwner; -import org.apache.flink.runtime.io.network.buffer.BufferProvider; -import org.apache.flink.runtime.io.network.partition.queue.IllegalQueueIteratorRequestException; -import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueue; -import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator; -import org.apache.flink.runtime.io.network.partition.queue.PipelinedPartitionQueue; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionType; -import org.apache.flink.runtime.jobgraph.JobID; -import org.apache.flink.runtime.messages.JobManagerMessages; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import scala.Option; - -import java.io.IOException; - -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.base.Preconditions.checkState; - -public class IntermediateResultPartition implements BufferPoolOwner { - - private static final Logger LOG = LoggerFactory.getLogger(IntermediateResultPartition.class); - - private final RuntimeEnvironment environment; - - /** - * Note: This index needs to correspond to the index of the partition in - * {@link ExecutionVertex#resultPartitions}, which might be a little - * fragile as the data availability notifications use it. - */ - private final int partitionIndex; - - private final JobID jobId; - - private final ExecutionAttemptID producerExecutionId; - - private final IntermediateResultPartitionID partitionId; - - private final IntermediateResultPartitionType partitionType; - - private final NetworkEnvironment networkEnvironment; - - private final IntermediateResultPartitionQueue[] queues; - - private volatile boolean hasNotifiedConsumers; - - private volatile boolean isReleased; - - private boolean isFinished; - - private BufferPool bufferPool; - - public IntermediateResultPartition(RuntimeEnvironment environment, int partitionIndex, JobID jobId, ExecutionAttemptID executionId, IntermediateResultPartitionID partitionId, IntermediateResultPartitionType partitionType, IntermediateResultPartitionQueue[] partitionQueues, NetworkEnvironment networkEnvironment) { - this.environment = environment; - this.partitionIndex = partitionIndex; - this.jobId = jobId; - this.producerExecutionId = executionId; - this.partitionId = partitionId; - this.partitionType = partitionType; - this.networkEnvironment = networkEnvironment; - this.queues = partitionQueues; - } - - // ------------------------------------------------------------------------ - // Properties - // ------------------------------------------------------------------------ - - public void setBufferPool(BufferPool bufferPool) { - checkArgument(bufferPool.getNumberOfRequiredMemorySegments() == getNumberOfQueues(), "Buffer pool has not enough buffers for this intermediate result."); - checkState(this.bufferPool == null, "Buffer pool has already been set for intermediate result partition."); - - this.bufferPool = checkNotNull(bufferPool); - } - - public ExecutionAttemptID getProducerExecutionId() { - return producerExecutionId; - } - - public IntermediateResultPartitionID getPartitionId() { - return partitionId; - } - - public JobID getJobId() { - return jobId; - } - - public int getNumberOfQueues() { - return queues.length; - } - - public BufferProvider getBufferProvider() { - return bufferPool; - } - - public boolean isFinished() { - return isFinished; - } - - // ------------------------------------------------------------------------ - // Produce - // ------------------------------------------------------------------------ - - public void add(Buffer buffer, int targetQueue) throws IOException { - synchronized (queues) { - if (isReleased) { - buffer.recycle(); - } - else { - checkInProducePhase(); - queues[targetQueue].add(buffer); - } - } - - maybeNotifyConsumers(partitionType.isPipelined()); - } - - public void finish() throws IOException { - boolean success = false; - - synchronized (queues) { - checkInProducePhase(); - - try { - if (!isReleased) { - for (IntermediateResultPartitionQueue queue : queues) { - queue.finish(); - } - - success = true; - } - } - finally { - isFinished = true; - } - } - - if (success) { - // Notify at this point in any case either because of the end - // of a blocking result or an empty pipelined result. - maybeNotifyConsumers(true); - - if (!partitionType.isPersistent() && bufferPool != null) { - // If this partition is not persistent, immediately destroy - // the buffer pool. For persistent intermediate results, the - // partition manager needs to release the buffer pool. - bufferPool.destroy(); - } - } - } - - public void releaseAllResources() throws IOException { - synchronized (queues) { - if (!isReleased) { - try { - for (IntermediateResultPartitionQueue queue : queues) { - try { - queue.discard(); - } - catch (Throwable t) { - LOG.error("Error while discarding queue: " + t.getMessage(), t); - } - } - - if (bufferPool != null) { - bufferPool.destroy(); - } - } - finally { - isReleased = true; - } - } - } - } - - // ------------------------------------------------------------------------ - // Consume - // ------------------------------------------------------------------------ - - public IntermediateResultPartitionQueueIterator getQueueIterator(int queueIndex, Optional bufferProvider) throws IOException { - synchronized (queues) { - if (isReleased) { - throw new IllegalQueueIteratorRequestException("Intermediate result partition has already been released."); - } - - if (queueIndex < 0 || queueIndex >= queues.length) { - throw new IllegalQueueIteratorRequestException("Illegal queue index: " + queueIndex + ", allowed: 0-" + (queues.length - 1)); - } - - return queues[queueIndex].getQueueIterator(bufferProvider); - } - } - - // ------------------------------------------------------------------------ - - @Override - public String toString() { - return "Intermediate result partition " + partitionId + " [num queues: " + queues.length + ", " + (isFinished ? "finished" : "not finished") + "]"; - } - - private void checkInProducePhase() { - checkState(!isReleased, "Partition has already been discarded."); - checkState(!isFinished, "Partition has already been finished."); - } - - /** - * Maybe notifies consumers of this result partition. - */ - private void maybeNotifyConsumers(boolean doNotify) throws IOException { - if (doNotify && !hasNotifiedConsumers) { - scheduleOrUpdateConsumers(); - hasNotifiedConsumers = true; - } - } - - private void scheduleOrUpdateConsumers() throws IOException { - while (!isReleased) { - final JobManagerMessages.ConsumerNotificationResult result = AkkaUtils.ask( - networkEnvironment.getJobManager(), - new JobManagerMessages.ScheduleOrUpdateConsumers(jobId, producerExecutionId, partitionIndex), - networkEnvironment.getJobManagerTimeout()); - - if (result.success()) { - return; - } - else { - Option error = result.error(); - if (error.isDefined()) { - throw new IOException(error.get().getMessage(), error.get()); - } - } - - try { - Thread.sleep(10); - } - catch (InterruptedException e) { - throw new IOException("Unexpected interruption during consumer schedule or update.", e); - } - } - } - - // ------------------------------------------------------------------------ - // Buffer pool owner methods - // ------------------------------------------------------------------------ - - /** - * If this partition is registered as the {@link BufferPoolOwner} of a - * {@link BufferPool}, it will forward the requests to the queues. - */ - @Override - public void recycleBuffers(int numBuffersToRecycle) throws IOException { - int numRecycledBuffers = 0; - - for (IntermediateResultPartitionQueue queue : queues) { - numRecycledBuffers += queue.recycleBuffers(); - - if (numRecycledBuffers >= numBuffersToRecycle) { - break; - } - } - } - - // ------------------------------------------------------------------------ - - public static IntermediateResultPartition create(RuntimeEnvironment environment, int partitionIndex, JobID jobId, ExecutionAttemptID executionId, NetworkEnvironment networkEnvironment, PartitionDeploymentDescriptor desc) { - final IntermediateResultPartitionID partitionId = checkNotNull(desc.getPartitionId()); - final IntermediateResultPartitionType partitionType = checkNotNull(desc.getPartitionType()); - - final IntermediateResultPartitionQueue[] partitionQueues = new IntermediateResultPartitionQueue[desc.getNumberOfQueues()]; - - // TODO The queues need to be created depending on the result type - for (int i = 0; i < partitionQueues.length; i++) { - partitionQueues[i] = new PipelinedPartitionQueue(); - } - - return new IntermediateResultPartition(environment, partitionIndex, jobId, executionId, partitionId, partitionType, partitionQueues, networkEnvironment); - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartitionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartitionManager.java deleted file mode 100644 index 55a741be1cc02..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartitionManager.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * 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; - -import com.google.common.base.Optional; -import com.google.common.collect.HashBasedTable; -import com.google.common.collect.Table; -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.io.network.buffer.BufferProvider; -import org.apache.flink.runtime.io.network.partition.queue.IllegalQueueIteratorRequestException; -import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -/** - * The intermediate result partition manager keeps track of all available - * partitions of a task manager and - */ -public class IntermediateResultPartitionManager implements IntermediateResultPartitionProvider { - - private static final Logger LOG = LoggerFactory.getLogger(IntermediateResultPartitionManager.class); - - public final Table partitions = HashBasedTable.create(); - - private boolean isShutdown; - - public void registerIntermediateResultPartition(IntermediateResultPartition partition) throws IOException { - synchronized (partitions) { - if (isShutdown) { - throw new IOException("Intermediate result partition manager has already been shut down."); - } - - if (partitions.put(partition.getProducerExecutionId(), partition.getPartitionId(), partition) != null) { - throw new IOException("Tried to re-register intermediate result partition."); - } - } - } - - public void failIntermediateResultPartitions(ExecutionAttemptID producerExecutionId) { - synchronized (partitions) { - List partitionsToFail = new ArrayList(); - - for (IntermediateResultPartitionID partitionId : partitions.row(producerExecutionId).keySet()) { - partitionsToFail.add(partitions.get(producerExecutionId, partitionId)); - } - - for(IntermediateResultPartition partition : partitionsToFail) { - failIntermediateResultPartition(partition); - } - } - } - - private void failIntermediateResultPartition(IntermediateResultPartition partition) { - if (partition != null) { - try { - partition.releaseAllResources(); - } - catch (Throwable t) { - LOG.error("Error during release of produced intermediate result partition: " + t.getMessage(), t); - } - } - } - - public void shutdown() { - synchronized (partitions) { - for (IntermediateResultPartition partition : partitions.values()) { - try { - partition.releaseAllResources(); - } - catch (IOException e) { - LOG.error("Error while releasing intermediate result partition: " + e.getMessage(), e); - } - } - - isShutdown = true; - } - } - - public int getNumberOfRegisteredPartitions() { - synchronized (partitions) { - return partitions.size(); - } - } - - // ------------------------------------------------------------------------ - // Intermediate result partition provider - // ------------------------------------------------------------------------ - - @Override - public IntermediateResultPartitionQueueIterator getIntermediateResultPartitionIterator( - ExecutionAttemptID producerExecutionId, - IntermediateResultPartitionID partitionId, - int queueIndex, - Optional bufferProvider) throws IOException { - - synchronized (partitions) { - IntermediateResultPartition partition = partitions.get(producerExecutionId, partitionId); - - if (partition == null) { - if (!partitions.containsRow(producerExecutionId)) { - throw new IllegalQueueIteratorRequestException("Unknown producer execution ID " + producerExecutionId + "."); - } - - throw new IllegalQueueIteratorRequestException("Unknown partition " + partitionId + "."); - } - - return partition.getQueueIterator(queueIndex, bufferProvider); - } - } -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/PipelinedPartitionQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java similarity index 72% rename from flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/PipelinedPartitionQueue.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java index a24cdeb24d23d..32be825f9e174 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/PipelinedPartitionQueue.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartition.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.runtime.io.network.partition.queue; +package org.apache.flink.runtime.io.network.partition; import com.google.common.base.Optional; import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; @@ -29,9 +29,9 @@ import java.util.Queue; /** - * An in-memory only queue, which can be consumed once by a single consumer. + * An in-memory only subpartition, which can be consumed once by a single consumer. */ -public class PipelinedPartitionQueue implements IntermediateResultPartitionQueue { +public class PipelinedSubpartition extends ResultSubpartition { final Queue queue = new ArrayDeque(); @@ -43,12 +43,18 @@ public class PipelinedPartitionQueue implements IntermediateResultPartitionQueue private boolean hasConsumer; + public PipelinedSubpartition(int index, ResultPartition parent) { + super(index, parent); + } + @Override public void add(Buffer buffer) { synchronized (queue) { if (!hasFinishedProduce) { queue.add(buffer); + updateStatistics(buffer); + maybeNotifyListener(); } else if (!buffer.isRecycled()) { @@ -80,13 +86,13 @@ public void finish() { } @Override - public int recycleBuffers() { + public int releaseMemory() { // Nothing to do here. Buffers are recycled when they are consumed. return 0; } @Override - public void discard() { + public void release() { synchronized (queue) { Buffer buffer; while ((buffer = queue.poll()) != null) { @@ -103,19 +109,19 @@ public void discard() { } @Override - public IntermediateResultPartitionQueueIterator getQueueIterator(Optional bufferProvider) throws IllegalQueueIteratorRequestException { + public ResultSubpartitionView getReadView(Optional bufferProvider) throws IllegalSubpartitionRequestException { synchronized (queue) { if (hasBeenDiscarded) { - throw new IllegalQueueIteratorRequestException("Queue has been discarded during produce phase."); + throw new IllegalSubpartitionRequestException("Queue has been discarded during produce phase."); } if (hasConsumer) { - throw new IllegalQueueIteratorRequestException("Consumable once queue has been consumed/is being consumed."); + throw new IllegalSubpartitionRequestException("Consumable once queue has been consumed/is being consumed."); } hasConsumer = true; - return new PipelinedPartitionQueueIterator(this); + return new PipelinedSubpartitionView(this); } } @@ -130,20 +136,35 @@ private void maybeNotifyListener() { } } - private static class PipelinedPartitionQueueIterator implements IntermediateResultPartitionQueueIterator { + private static class PipelinedSubpartitionView implements ResultSubpartitionView { - private final PipelinedPartitionQueue partitionQueue; + private final PipelinedSubpartition partitionQueue; private boolean isDiscarded; - private PipelinedPartitionQueueIterator(PipelinedPartitionQueue partitionQueue) { + private volatile boolean isConsumed; + + private PipelinedSubpartitionView(PipelinedSubpartition partitionQueue) { this.partitionQueue = partitionQueue; } @Override public boolean isConsumed() { + if (isConsumed) { + return true; + } + synchronized (partitionQueue.queue) { - return (partitionQueue.isFinished() && partitionQueue.queue.isEmpty()) || partitionQueue.hasBeenDiscarded; + isConsumed = (partitionQueue.isFinished() && partitionQueue.queue.isEmpty()) || partitionQueue.hasBeenDiscarded; + } + + if (isConsumed) { + partitionQueue.notifyConsumed(); + + return true; + } + else { + return false; } } @@ -155,10 +176,10 @@ public Buffer getNextBuffer() { } @Override - public void discard() { + public void release() { synchronized (partitionQueue.queue) { if (!isDiscarded) { - partitionQueue.discard(); + partitionQueue.release(); isDiscarded = true; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java new file mode 100644 index 0000000000000..a43a85e428cf4 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartition.java @@ -0,0 +1,482 @@ +/* + * 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; + +import akka.actor.ActorRef; +import com.google.common.base.Optional; +import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.executiongraph.ExecutionVertex; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferPool; +import org.apache.flink.runtime.io.network.buffer.BufferPoolOwner; +import org.apache.flink.runtime.io.network.buffer.BufferProvider; +import org.apache.flink.runtime.io.network.netty.PartitionRequestClient; +import org.apache.flink.runtime.io.network.partition.consumer.LocalInputChannel; +import org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel; +import org.apache.flink.runtime.jobgraph.JobID; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; +import org.apache.flink.runtime.messages.JobManagerMessages; +import org.apache.flink.runtime.taskmanager.TaskManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Option; +import scala.concurrent.duration.FiniteDuration; + +import java.io.IOException; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +/** + * A result partition for data produced by a single task. There is a single producing task for each + * result partition, but each producing task can produce more than one partition. + *

+ * Essentially, a result partition is a collection of {@link Buffer}. The buffers are organized in + * one or more partitions, which allow to further partition the data depending on the number of + * consuming task. + *

+ * These partitions have different characteristics depending on the {@link ResultPartitionType}. + * Tasks, which consume a result partition have to request one of its partitions. The request + * happens either remotely (see {@link RemoteInputChannel} and {@link PartitionRequestClient}) or + * locally (see {@link LocalInputChannel} and {@link ResultPartitionManager}). + * + *

Life-cycle

+ * + * The life-cycle of each result partition has three (possibly overlapping) phases: + *
    + *
  1. Produce:
  2. + *
  3. Consume:
  4. + *
  5. Release:
  6. + *
+ * + *

Lazy deployment and updates of consuming tasks

+ * + * Before a consuming task can request the result, it has to be deployed. The time of deployment + * depends on the PIPELINED vs. BLOCKING characteristic of the result partition. With pipelined + * results, receivers are deployed as soon as the first buffer is added to the result partition. + * With blocking results on the other hand, receivers are deployed after the partition is finished. + * + *

Buffer management

+ * + *

State management

+ */ +public class ResultPartition implements BufferPoolOwner { + + private static final Logger LOG = LoggerFactory.getLogger(ResultPartition.class); + + /** + * Partition index corresponding to the index of the partition in + * {@link ExecutionVertex#resultPartitions}. + * + * TODO Use different identifier as the index is error-prone. + */ + private final int index; + + private final JobID jobId; + + /** Execution ID of the task producing this partition */ + private final ExecutionAttemptID executionId; + + /** ID of this partition */ + private final ResultPartitionID partitionId; + + /** Type of this partition. Defines the concrete subpartition implementation to use. */ + private final ResultPartitionType partitionType; + + /** The partitions of this partition. At least one. */ + private final ResultSubpartition[] partitions; + + /** The partition manager */ + private final ResultPartitionManager partitionManager; + + private final ActorRef jobManager; + + private final FiniteDuration jobManagerTimeout; + + // - Runtime state -------------------------------------------------------- + + private BufferPool bufferPool; + + private volatile boolean notifiedConsumersFlag; + + private boolean finishedFlag; + + /** + * The total number of references to partitions of this result. The result partition can be + * safely released, iff the reference count is zero. A reference count of -1 denotes that the + * result partition has been released. + */ + private int referenceCount; + + // - Statistics ---------------------------------------------------------- + + /** The total number of buffers (both data and event buffers) */ + private int totalNumberOfBuffers; + + /** The total number of bytes (both data and event buffers) */ + private long totalNumberOfBytes; + + public ResultPartition(int index, JobID jobId, ExecutionAttemptID executionId, ResultPartitionID partitionId, ResultPartitionType partitionType, int numberOfSubpartitions, NetworkEnvironment networkEnvironment, IOManager ioManager) { + checkArgument(index >= 0); + this.index = index; + this.jobId = checkNotNull(jobId); + this.executionId = checkNotNull(executionId); + this.partitionId = checkNotNull(partitionId); + this.partitionType = checkNotNull(partitionType); + this.partitions = new ResultSubpartition[numberOfSubpartitions]; + this.partitionManager = checkNotNull(networkEnvironment.getPartitionManager()); + this.jobManager = checkNotNull(networkEnvironment.getJobManager()); + this.jobManagerTimeout = checkNotNull(networkEnvironment.getJobManagerTimeout()); + + // Create the subpartitions. Initially, the partition needs to be consumed at least once + // before it can be released. + synchronized (partitions) { + switch (partitionType) { + case BLOCKING: + for (int i = 0; i < partitions.length; i++) { + partitions[i] = new SpillableSubpartition(i, this, ioManager); + } + + break; + + case PIPELINED: + for (int i = 0; i < partitions.length; i++) { + partitions[i] = new PipelinedSubpartition(i, this); + } + + break; + + default: + throw new IllegalArgumentException("Unsupported result partition type."); + } + + // Initially set the reference count to ensure that the partition has to be consumed + // once before it can be released. + this.referenceCount = partitions.length; + } + } + + /** + * Registers a buffer pool with this result partition. + *

+ * There is one pool for each result partition, which is shared by all its sub partitions. + *

+ * The pool is registered with the partition *after* it as been constructed in order to conform + * to the life-cycle of task registrations in the {@link TaskManager}. + */ + public void registerBufferPool(BufferPool bufferPool) { + checkArgument(bufferPool.getNumberOfRequiredMemorySegments() == getNumberOfSubpartitions(), "Bug in result partition setup logic: Buffer pool has not enough guaranteed buffers for this result partition."); + checkState(this.bufferPool == null, "Bug in result partition setup logic: Already registered buffer pool."); + + this.bufferPool = checkNotNull(bufferPool); + + // If the partition type has no back pressure, we register with the buffer pool for + // callbacks to release memory. + if (!partitionType.hasBackPressure()) { + bufferPool.setBufferPoolOwner(this); + } + } + + public JobID getJobId() { + return jobId; + } + + public ExecutionAttemptID getExecutionId() { + return executionId; + } + + public ResultPartitionID getPartitionId() { + return partitionId; + } + + public int getNumberOfSubpartitions() { + return partitions.length; + } + + public BufferProvider getBufferProvider() { + return bufferPool; + } + + public int getTotalNumberOfBuffers() { + return totalNumberOfBuffers; + } + + public long getTotalNumberOfBytes() { + return totalNumberOfBytes; + } + + // ------------------------------------------------------------------------ + + /** + * Adds a buffer to the subpartition with the given index. + *

+ * For PIPELINED results, this will trigger the deployment of consuming tasks after the first + * buffer has been added. + */ + public void add(Buffer buffer, int subpartitionIndex) throws IOException { + boolean success = false; + + try { + synchronized (partitions) { + checkInProduceState(); + + partitions[subpartitionIndex].add(buffer); + + // Update statistics + totalNumberOfBuffers++; + totalNumberOfBytes += buffer.getSize(); + + success = true; + } + } + finally { + if (success) { + if (partitionType.isPipelined()) { + notifyConsumersOnce(); + } + } + else { + buffer.recycle(); + } + } + } + + /** + * Finishes the result partition. + *

+ * After this operation, it is not possible to add further data to the result partition. + *

+ * For BLOCKING results, this will trigger the deployment of consuming tasks. + */ + public void finish() throws IOException { + boolean success = false; + + try { + synchronized (partitions) { + checkInProduceState(); + + for (ResultSubpartition partition : partitions) { + partition.finish(); + } + + success = true; + } + } + finally { + if (success) { + finishedFlag = true; + + notifyConsumersOnce(); + } + } + } + + /** + * Returns the requested subpartition. + */ + public ResultSubpartitionView getSubpartition(int index, Optional bufferProvider) throws IOException { + synchronized (partitions) { + checkState(!hasBeenReleased(), "Result partition already released."); + checkState(referenceCount > 0, "Result partition needs to be pinned, before a subpartition can be consumed."); + + return partitions[index].getReadView(bufferProvider); + } + } + + /** + * Releases the result partition. + */ + public void release() { + synchronized (partitions) { + // Only release the partition if there is currently no reference to it. + if (referenceCount == 0) { + try { + for (ResultSubpartition partition : partitions) { + try { + partition.release(); + } + // Catch this in order to ensure that release is called on subpartitions + catch (Throwable t) { + LOG.error("Error during release of result subpartition: " + t.getMessage(), t); + } + } + } + finally { + referenceCount = -1; + + if (bufferPool != null) { + bufferPool.lazyDestroy(); + } + } + } + else if (referenceCount > 0) { + throw new IllegalStateException("Bug in result partition release logic: pending references."); + } + else { + throw new IllegalStateException("Bug in result partition release logic: already released."); + } + } + } + + /** + * Releases buffers held by this result partition. + *

+ * This is a callback from the buffer pool, which is registered for result partitions, which + * are back pressure-free. + */ + @Override + public void releaseMemory(int toRelease) throws IOException { + checkArgument(toRelease > 0); + + for (ResultSubpartition partition : partitions) { + toRelease -= partition.releaseMemory(); + + // Only release + if (toRelease <= 0) { + break; + } + } + + int numRecycledBuffers = 0; + + for (ResultSubpartition queue : partitions) { + numRecycledBuffers += queue.releaseMemory(); + + if (numRecycledBuffers >= toRelease) { + break; + } + } + } + + @Override + public String toString() { + return executionId + ":" + partitionId + ":" + index + "[" + partitionType + "]"; + } + + // ------------------------------------------------------------------------ + + /** + * Pins the result partition. + *

+ * The partition can only be released after each subpartition has been consumed once per pin + * operation. + */ + void pin() { + synchronized (partitions) { + referenceCount += partitions.length; + } + } + + /** + * Unpins the result partition. + *

+ * The partition can be released after this operation (if there is no pinning in between). + */ + void unpin() { + synchronized (partitions) { + referenceCount = 0; + } + } + + /** + * Notification when a subpartition is released. + */ + void onConsumedSubpartition(int subpartitionIndex) { + synchronized (partitions) { + referenceCount--; + + if (referenceCount == 0) { + partitionManager.onConsumedPartition(this); + } + else if (referenceCount < 0) { + throw new IllegalStateException("Bug in result subpartition release logic: decremented the reference count below 0."); + } + + if (LOG.isDebugEnabled() && referenceCount == 0) { + LOG.debug("Reference count for result partition reached zero."); + } + } + } + + boolean hasConsumer() { + synchronized (partitions) { + return referenceCount > 0; + } + } + + boolean hasBeenReleased() { + synchronized (partitions) { + return referenceCount < 0; + } + } + + // ------------------------------------------------------------------------ + + // + // IMPORTANT: Needs to be called from synchronized scope (guarded by "partitions") + // + private void checkInProduceState() { + checkState(!finishedFlag, "Partition already finished."); + checkState(referenceCount >= 0, "Partition already released."); + } + + /** + * Notifies consumers of this result partition once. + * + * TODO Use Akka's asynchronous tell call. As long as this is a blocking call, watch out to NOT + * call this method in synchronized scope (guarded by "partitions"). Otherwise, this will + * lead to deadlocks. + */ + private void notifyConsumersOnce() throws IOException { + if (!notifiedConsumersFlag) { + while (true) { + if (Thread.interrupted()) { + return; + } + + final JobManagerMessages.ConsumerNotificationResult result = AkkaUtils.ask( + jobManager, + new JobManagerMessages.ScheduleOrUpdateConsumers(jobId, executionId, index), + jobManagerTimeout); + + if (result.success()) { + notifiedConsumersFlag = true; + return; + } + else { + Option error = result.error(); + if (error.isDefined()) { + throw new IOException(error.get().getMessage(), error.get()); + } + } + + try { + Thread.sleep(10); + } + catch (InterruptedException e) { + throw new IOException("Unexpected interruption during consumer schedule or update.", e); + } + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java new file mode 100644 index 0000000000000..d4142a75836e5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java @@ -0,0 +1,113 @@ +/* + * 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; + +import com.google.common.base.Optional; +import com.google.common.collect.HashBasedTable; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Table; +import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; +import org.apache.flink.runtime.io.network.buffer.BufferProvider; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Map; + +import static com.google.common.base.Preconditions.checkState; + +/** + * The result partition manager keeps track of all currently produced/consumed partitions of a + * task manager. + */ +public class ResultPartitionManager implements ResultPartitionProvider { + + private static final Logger LOG = LoggerFactory.getLogger(ResultPartitionManager.class); + + public final Table partitions = HashBasedTable.create(); + + private boolean isShutdown; + + public void registerIntermediateResultPartition(ResultPartition partition) throws IOException { + synchronized (partitions) { + checkState(!isShutdown, "Result partition manager already shut down."); + + ResultPartition previous = partitions.put(partition.getExecutionId(), partition.getPartitionId(), partition); + + if (previous != null) { + throw new IllegalStateException("Result partition already registered."); + } + } + } + + @Override + public ResultSubpartitionView getSubpartition(ExecutionAttemptID executionId, ResultPartitionID partitionId, int subpartitionIndex, Optional bufferProvider) throws IOException { + synchronized (partitions) { + ResultPartition partition = partitions.get(executionId, partitionId); + + if (partition == null) { + throw new IOException(String.format("Unknown partition %s:%s.", executionId, partitionId)); + } + + return partition.getSubpartition(subpartitionIndex, bufferProvider); + } + } + + public void releasePartitionsProducedBy(ExecutionAttemptID executionId) { + synchronized (partitions) { + Map row = partitions.row(executionId); + + for (ResultPartition partition : partitions.row(executionId).values()) { + partition.unpin(); + + partition.release(); + } + + for (ResultPartitionID partitionId : ImmutableList.copyOf(row.keySet())) { + partitions.remove(executionId, partitionId); + } + } + } + + public void shutdown() { + synchronized (partitions) { + for (ResultPartition partition : partitions.values()) { + partition.release(); + } + + partitions.clear(); + + isShutdown = true; + } + } + + // ------------------------------------------------------------------------ + // Notifications + // ------------------------------------------------------------------------ + + void onConsumedPartition(ResultPartition partition) { + synchronized (partitions) { + // TODO Add historic intermediate result manager, where this partition can live on + partitions.remove(partition.getExecutionId(), partition.getPartitionId()); + + partition.release(); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartitionProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionProvider.java similarity index 68% rename from flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartitionProvider.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionProvider.java index b18b3fcf09f8b..4552e2dac7f89 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/IntermediateResultPartitionProvider.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionProvider.java @@ -21,17 +21,15 @@ import com.google.common.base.Optional; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.network.buffer.BufferProvider; -import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; import java.io.IOException; -public interface IntermediateResultPartitionProvider { +public interface ResultPartitionProvider { - IntermediateResultPartitionQueueIterator getIntermediateResultPartitionIterator( - ExecutionAttemptID producerExecutionId, - IntermediateResultPartitionID partitionId, - int requestedQueueIndex, - Optional bufferProvider) throws IOException; + /** + * Returns the requested intermediate result partition input view. + */ + ResultSubpartitionView getSubpartition(ExecutionAttemptID executionId, ResultPartitionID partitionId, int index, Optional bufferProvider) throws IOException; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionType.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionType.java similarity index 74% rename from flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionType.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionType.java index 25d057ed32f1c..222a29f4a8d4a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionType.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionType.java @@ -16,22 +16,29 @@ * limitations under the License. */ -package org.apache.flink.runtime.jobgraph; +package org.apache.flink.runtime.io.network.partition; -public enum IntermediateResultPartitionType { +public enum ResultPartitionType { BLOCKING(true, false, false), PIPELINED(false, true, true), + PIPELINED_PERSISTENT(true, true, true); + /** Does the partition live longer than the consuming task? */ private final boolean isPersistent; + /** Can the partition be consumed while being produced? */ private final boolean isPipelined; + /** Does the partition produce back pressure when not consumed? */ private final boolean hasBackPressure; - IntermediateResultPartitionType(boolean isPersistent, boolean isPipelined, boolean hasBackPressure) { + /** + * Specifies the behaviour of an intermediate result partition at runtime. + */ + ResultPartitionType(boolean isPersistent, boolean isPipelined, boolean hasBackPressure) { this.isPersistent = isPersistent; this.isPipelined = isPipelined; this.hasBackPressure = hasBackPressure; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java new file mode 100644 index 0000000000000..d5da5fa8404b8 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java @@ -0,0 +1,67 @@ +/* + * 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; + +import com.google.common.base.Optional; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferProvider; + +import java.io.IOException; + +public abstract class ResultSubpartition { + + protected final int index; + + protected final ResultPartition parent; + + // - Statistics ---------------------------------------------------------- + + /** The total number of buffers (both data and event buffers) */ + private int totalNumberOfBuffers; + + /** The total number of bytes (both data and event buffers) */ + private long totalNumberOfBytes; + + public ResultSubpartition(int index, ResultPartition parent) { + this.index = index; + this.parent = parent; + } + + protected void updateStatistics(Buffer buffer) { + totalNumberOfBuffers++; + totalNumberOfBytes += buffer.getSize(); + } + + protected void notifyConsumed() { + parent.onConsumedSubpartition(index); + } + + abstract void add(Buffer buffer) throws IOException; + + abstract void finish() throws IOException; + + abstract void release() throws IOException; + + abstract ResultSubpartitionView getReadView(Optional bufferProvider) throws IOException; + + abstract boolean isFinished(); + + abstract int releaseMemory() throws IOException; + +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IntermediateResultPartitionQueueIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionView.java similarity index 91% rename from flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IntermediateResultPartitionQueueIterator.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionView.java index 9fbb331d34147..d9ae0f56ca38d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IntermediateResultPartitionQueueIterator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartitionView.java @@ -16,14 +16,14 @@ * limitations under the License. */ -package org.apache.flink.runtime.io.network.partition.queue; +package org.apache.flink.runtime.io.network.partition; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.util.event.NotificationListener; import java.io.IOException; -public interface IntermediateResultPartitionQueueIterator { +public interface ResultSubpartitionView { // ------------------------------------------------------------------------ // Properties @@ -52,18 +52,18 @@ public interface IntermediateResultPartitionQueueIterator { */ Buffer getNextBuffer() throws IOException; - /** - * Discards the consumption of this queue iterator. - */ - void discard() throws IOException; - /** * Subscribes to data availability notifications. *

* Returns whether the subscription was successful. A subscription fails, * if there is data available. */ - boolean subscribe(NotificationListener listener) throws AlreadySubscribedException; + boolean subscribe(NotificationListener listener) throws IOException; + + /** + * Discards the consumption of this queue iterator. + */ + void release() throws IOException; // ------------------------------------------------------------------------ diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java new file mode 100644 index 0000000000000..f85692ab99586 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java @@ -0,0 +1,254 @@ +/* + * 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; + +import com.google.common.base.Optional; +import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent; +import org.apache.flink.runtime.io.network.api.serialization.EventSerializer; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferProvider; +import org.apache.flink.runtime.util.event.NotificationListener; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static com.google.common.base.Preconditions.checkNotNull; + +public class SpillableSubpartition extends ResultSubpartition { + + private final IOManager ioManager; + + private final List buffers = new ArrayList(); + + private volatile BufferFileWriter writer; + + private volatile boolean finishedProduce; + + public SpillableSubpartition(int index, ResultPartition parent, IOManager ioManager) { + super(index, parent); + this.ioManager = ioManager; + } + + @Override + public void add(Buffer buffer) throws IOException { + checkNotNull(buffer); + + synchronized (buffers) { + if (writer == null) { + buffers.add(buffer); + return; + } + } + + // else: spilling + writer.writeBlock(buffer); + } + + @Override + public void finish() throws IOException { + if (finishedProduce) { + // TODO Or rather throw an exception as this might indicate misuse? + return; + } + + add(EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE)); + + // If we already spilled, make sure that the writer is finished before we change our state + // in order to ensure that consumers don't request too early. + if (writer != null) { + synchronized (buffers) { + writer.close(); + } + } + + finishedProduce = true; + } + + @Override + public void release() throws IOException { + + } + + @Override + public ResultSubpartitionView getReadView(Optional bufferProvider) throws IOException { + if (!finishedProduce) { + throw new IllegalSubpartitionRequestException("Queue has not been finished yet."); + } + + if (!bufferProvider.isPresent()) { + throw new IllegalSubpartitionRequestException("Did not provide a buffer provider request, which is necessary at the moment."); + } + + synchronized (buffers) { + if (writer != null && writer.getNumberOfOutstandingRequests() == 0) { + return new SpilledSubpartitionView(this, ioManager, writer.getChannelID(), bufferProvider.get()); + } + else { + return new SpillableSubpartitionView(this, bufferProvider.get()); + } + } + } + + @Override + public boolean isFinished() { + return finishedProduce; + } + + @Override + public int releaseMemory() throws IOException { + if (writer == null) { + synchronized (buffers) { + writer = ioManager.createBufferFileWriter(ioManager.createChannel()); + + int numBuffers = buffers.size(); + + for (int i = 0; i < numBuffers; i++) { + writer.writeBlock(buffers.remove(0)); + } + + return numBuffers; + } + } + + return 0; + } + + boolean isInMemory() { + return writer == null; + } + +// ------------------------------------------------------------------------ + + static class SpillableSubpartitionView implements ResultSubpartitionView { + + private final SpillableSubpartition queue; + + private final BufferProvider bufferProvider; + + private final int numBuffers; + + private SpilledSubpartitionView reader; + + private int currentQueuePosition; + + private long currentBytesRead; + + private volatile boolean isConsumed; + + public SpillableSubpartitionView(SpillableSubpartition queue, BufferProvider bufferProvider) { + this.queue = queue; + this.bufferProvider = bufferProvider; + this.numBuffers = queue.buffers.size(); + } + + @Override + public boolean isConsumed() { + if (isConsumed) { + return true; + } + + synchronized (queue.buffers) { + if (reader != null) { + isConsumed = reader.isConsumed(); + } + else { + isConsumed = currentQueuePosition >= numBuffers; + } + } + + if (isConsumed) { + queue.notifyConsumed(); + + return true; + } + else { + return false; + } + } + + @Override + public Buffer getNextBuffer() throws IOException { + // 1) In-memory + synchronized (queue.buffers) { + if (queue.writer == null) { + if (currentQueuePosition < numBuffers) { + Buffer buffer = queue.buffers.get(currentQueuePosition); + + // Ensure that a concurrent spilling does not lead to a buffer leak + buffer.retain(); + + // TODO Fix hard coding of 8 bytes for the header + currentBytesRead += buffer.getSize() + 8; + currentQueuePosition++; + + return buffer; + } + + return null; + } + } + + // 2) Spilled + if (reader != null) { + return reader.getNextBuffer(); + } + + // 3) Spilling + // Make sure that all buffers are written before consuming them. We can't block here, + // because this might be called from an network I/O thread. + if (queue.writer.getNumberOfOutstandingRequests() > 0) { + return null; + } + + if (reader == null) { + reader = new SpilledSubpartitionView(queue, queue.ioManager, queue.writer.getChannelID(), bufferProvider, currentBytesRead); + } + + return reader.getNextBuffer(); + } + + @Override + public void release() throws IOException { + + } + + @Override + public boolean subscribe(NotificationListener listener) throws IOException { + if (reader == null) { + synchronized (queue.buffers) { + if (queue.writer == null) { + return false; + } + } + + // Spilling + if (queue.writer.getNumberOfOutstandingRequests() > 0) { + return queue.writer.subscribe(listener); + } + + return false; + } + + return reader.subscribe(listener); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java new file mode 100644 index 0000000000000..feb426ad4cc36 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionView.java @@ -0,0 +1,194 @@ +/* + * 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; + +import org.apache.flink.runtime.io.disk.iomanager.BufferFileReader; +import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.RequestDoneCallback; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferProvider; +import org.apache.flink.runtime.util.event.NotificationListener; + +import java.io.IOException; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +public class SpilledSubpartitionView implements ResultSubpartitionView { + + private final Object listenerLock = new Object(); + + private final ResultSubpartition partition; + + private final BufferFileReader reader; + + private final BufferProvider bufferProvider; + + private final Queue returnedBuffers = new ConcurrentLinkedQueue(); + + private volatile NotificationListener listener; + + private volatile Throwable ioError; + + private volatile boolean readerIsConsumed; + + private volatile boolean isConsumed; + + public SpilledSubpartitionView(ResultSubpartition partition, IOManager ioManager, FileIOChannel.ID channelId, BufferProvider bufferProvider) throws IOException { + this(partition, ioManager, channelId, bufferProvider, 0); + } + + public SpilledSubpartitionView(ResultSubpartition partition, IOManager ioManager, FileIOChannel.ID channelId, BufferProvider bufferProvider, long initialPosition) throws IOException { + this.partition = checkNotNull(partition); + this.reader = ioManager.createBufferFileReader(channelId, new ReadDoneCallback(this)); + this.bufferProvider = checkNotNull(bufferProvider); + + checkArgument(initialPosition >= 0, "Initial position is < 0."); + + if (initialPosition > 0) { + reader.seekToPosition(initialPosition); + } + + sendBufferReadRequests(); + } + + @Override + public boolean isConsumed() { + if (isConsumed) { + return true; + } + + isConsumed = readerIsConsumed && returnedBuffers.isEmpty(); + + if (isConsumed) { + partition.notifyConsumed(); + + return true; + } + + return false; + } + + @Override + public Buffer getNextBuffer() throws IOException { + checkError(); + + Buffer buffer = returnedBuffers.poll(); + + if (buffer == null && !isConsumed()) { + sendBufferReadRequests(); + + // Try again... + buffer = returnedBuffers.poll(); + } + + return buffer; + } + + @Override + public void release() throws IOException { + + } + + @Override + public boolean subscribe(NotificationListener listener) throws IOException { + checkError(); + + synchronized (listenerLock) { + if (isConsumed() || !returnedBuffers.isEmpty()) { + return false; + } + + if (this.listener == null) { + this.listener = listener; + return true; + } + } + + throw new AlreadySubscribedException(); + } + + private void maybeNotifyListener() { + synchronized (listenerLock) { + if (listener != null) { + final NotificationListener consumer = listener; + listener = null; + consumer.onNotification(); + } + } + } + + private void checkError() throws IOException { + if (ioError != null) { + throw new IOException(ioError.getMessage(), ioError); + } + } + + // ------------------------------------------------------------------------ + + private void sendBufferReadRequests() throws IOException { + Buffer buffer; + while ((buffer = bufferProvider.requestBuffer()) != null) { + reader.readInto(buffer); + } + } + + private static class ReadDoneCallback implements RequestDoneCallback { + + private final SpilledSubpartitionView queue; + + public ReadDoneCallback(SpilledSubpartitionView queue) { + this.queue = queue; + } + + @Override + public void requestSuccessful(Buffer buffer) { + if (!queue.readerIsConsumed) { + synchronized (queue.listenerLock) { + queue.returnedBuffers.add(buffer); + + if (queue.reader.isConsumed()) { + queue.readerIsConsumed = true; + } + + queue.maybeNotifyListener(); + } + } + else { + buffer.recycle(); + } + } + + @Override + public void requestFailed(Buffer buffer, IOException error) { + synchronized (queue.listenerLock) { + if (queue.ioError != null) { + queue.ioError = error; + } + + buffer.recycle(); + + queue.maybeNotifyListener(); + } + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java index ea8f459db1f43..30106c12bbb87 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannel.java @@ -22,13 +22,13 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.network.api.reader.BufferReader; import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueue; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultSubpartition; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; import java.io.IOException; /** - * An input channel is the consumer of a single queue of an {@link IntermediateResultPartitionQueue}. + * An input channel is the consumer of a single queue of an {@link ResultSubpartition}. *

* For each channel, the consumption life cycle is as follows: *

    @@ -43,11 +43,11 @@ public abstract class InputChannel { protected final ExecutionAttemptID producerExecutionId; - protected final IntermediateResultPartitionID partitionId; + protected final ResultPartitionID partitionId; protected final BufferReader reader; - protected InputChannel(int channelIndex, ExecutionAttemptID producerExecutionId, IntermediateResultPartitionID partitionId, BufferReader reader) { + protected InputChannel(int channelIndex, ExecutionAttemptID producerExecutionId, ResultPartitionID partitionId, BufferReader reader) { this.channelIndex = channelIndex; this.producerExecutionId = producerExecutionId; this.partitionId = partitionId; @@ -66,7 +66,7 @@ public ExecutionAttemptID getProducerExecutionId() { return producerExecutionId; } - public IntermediateResultPartitionID getPartitionId() { + public ResultPartitionID getPartitionId() { return partitionId; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java index 150aaea7dc9c0..1a5a2f2d3a942 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannel.java @@ -23,8 +23,8 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.network.api.reader.BufferReader; import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; import org.apache.flink.runtime.util.event.NotificationListener; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,13 +40,13 @@ public class LocalInputChannel extends InputChannel implements NotificationListe private static final Logger LOG = LoggerFactory.getLogger(LocalInputChannel.class); - private IntermediateResultPartitionQueueIterator queueIterator; + private ResultSubpartitionView queueIterator; private boolean isReleased; private Buffer lookAhead; - public LocalInputChannel(int channelIndex, ExecutionAttemptID producerExecutionId, IntermediateResultPartitionID partitionId, BufferReader reader) { + public LocalInputChannel(int channelIndex, ExecutionAttemptID producerExecutionId, ResultPartitionID partitionId, BufferReader reader) { super(channelIndex, producerExecutionId, partitionId, reader); } @@ -62,7 +62,11 @@ public void requestIntermediateResultPartition(int queueIndex) throws IOExceptio } queueIterator = reader.getIntermediateResultPartitionProvider() - .getIntermediateResultPartitionIterator(producerExecutionId, partitionId, queueIndex, Optional.of(reader.getBufferProvider())); + .getSubpartition(producerExecutionId, partitionId, queueIndex, Optional.of(reader.getBufferProvider())); + + if (queueIterator == null) { + throw new IOException("Error requesting sub partition."); + } getNextLookAhead(); } @@ -70,7 +74,11 @@ public void requestIntermediateResultPartition(int queueIndex) throws IOExceptio @Override public Buffer getNextBuffer() throws IOException { - checkState(queueIterator != null, "Queried for a buffer before requesting a queue."); + if (queueIterator == null) { + + checkState(queueIterator != null, "Queried for a buffer before requesting a queue."); + } + // After subscribe notification if (lookAhead == null) { @@ -120,7 +128,7 @@ public void releaseAllResources() throws IOException { } if (queueIterator != null) { - queueIterator.discard(); + queueIterator.release(); queueIterator = null; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java index 616a8a5decd3e..d204268a40e16 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java @@ -25,7 +25,7 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferProvider; import org.apache.flink.runtime.io.network.netty.PartitionRequestClient; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,7 +61,7 @@ public class RemoteInputChannel extends InputChannel { public RemoteInputChannel( int channelIndex, ExecutionAttemptID producerExecutionId, - IntermediateResultPartitionID partitionId, + ResultPartitionID partitionId, BufferReader reader, RemoteAddress producerAddress) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java index f1fae895a9319..983f168747f0c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/UnknownInputChannel.java @@ -23,7 +23,7 @@ import org.apache.flink.runtime.io.network.RemoteAddress; import org.apache.flink.runtime.io.network.api.reader.BufferReader; import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; import java.io.IOException; @@ -35,7 +35,7 @@ */ public class UnknownInputChannel extends InputChannel { - public UnknownInputChannel(int channelIndex, ExecutionAttemptID producerExecutionId, IntermediateResultPartitionID partitionId, BufferReader reader) { + public UnknownInputChannel(int channelIndex, ExecutionAttemptID producerExecutionId, ResultPartitionID partitionId, BufferReader reader) { super(channelIndex, producerExecutionId, partitionId, reader); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IntermediateResultPartitionQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IntermediateResultPartitionQueue.java deleted file mode 100644 index 278e2a1c807d7..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/queue/IntermediateResultPartitionQueue.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.queue; - -import com.google.common.base.Optional; -import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.buffer.BufferProvider; - -import java.io.IOException; - -public interface IntermediateResultPartitionQueue { - - // ------------------------------------------------------------------------ - // Produce - // ------------------------------------------------------------------------ - - void add(Buffer buffer) throws IOException; - - void finish() throws IOException; - - void discard() throws IOException; - - // ------------------------------------------------------------------------ - // Consume - // ------------------------------------------------------------------------ - - IntermediateResultPartitionQueueIterator getQueueIterator(Optional bufferProvider) throws IllegalQueueIteratorRequestException, IOException; - - // ------------------------------------------------------------------------ - // Properties - // ------------------------------------------------------------------------ - - boolean isFinished(); - - int recycleBuffers() throws IOException; - -} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java index 5896dcf3a0ab7..ca1ade90ff719 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java @@ -46,7 +46,7 @@ public class SerializedUpdateBuffer extends AbstractPagedOutputView { private ArrayDeque fullBuffers; - private BlockChannelWriter currentWriter; + private BlockChannelWriter currentWriter; private final IOManager ioManager; @@ -182,7 +182,7 @@ public ReadEnd switchBuffers() throws IOException { // now close the writer and create the reader currentWriter.close(); - final BlockChannelReader reader = ioManager.createBlockChannelReader(currentWriter.getChannelID()); + final BlockChannelReader reader = ioManager.createBlockChannelReader(currentWriter.getChannelID()); // gather some memory segments to circulate while reading back the data final List readSegments = new ArrayList(); @@ -263,14 +263,14 @@ private static final class ReadEnd extends AbstractPagedInputView { private final Deque fullBufferSource; - private final BlockChannelReader spilledBufferSource; + private final BlockChannelReader spilledBufferSource; private int spilledBuffersRemaining; private int requestsRemaining; private ReadEnd(MemorySegment firstMemSegment, LinkedBlockingQueue emptyBufferTarget, - Deque fullBufferSource, BlockChannelReader spilledBufferSource, + Deque fullBufferSource, BlockChannelReader spilledBufferSource, List emptyBuffers, int numBuffersSpilled) throws IOException { super(firstMemSegment, firstMemSegment.getInt(0), HEADER_LENGTH); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java index b9e1eac181631..99844aded79bc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java @@ -24,6 +24,7 @@ import org.apache.commons.lang3.Validate; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.InputSplitSource; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; @@ -320,7 +321,11 @@ public IntermediateDataSet createAndAddResultDataSet() { } public IntermediateDataSet createAndAddResultDataSet(IntermediateDataSetID id) { - IntermediateDataSet result = new IntermediateDataSet(id, this); + return createAndAddResultDataSet(id, ResultPartitionType.PIPELINED); + } + + public IntermediateDataSet createAndAddResultDataSet(IntermediateDataSetID id, ResultPartitionType runtimeType) { + IntermediateDataSet result = new IntermediateDataSet(id, runtimeType, this); this.results.add(result); return result; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java index c8415dee2bc87..9dc20d6680dbc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java @@ -18,9 +18,13 @@ package org.apache.flink.runtime.jobgraph; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; + import java.util.ArrayList; import java.util.List; +import static com.google.common.base.Preconditions.checkNotNull; + /** * An intermediate data set is the data set produced by an operator - either a * source or any intermediate operation. @@ -38,6 +42,9 @@ public class IntermediateDataSet implements java.io.Serializable { private final AbstractJobVertex producer; // the operation that produced this data set private final List consumers = new ArrayList(); + + // The type of partition to use at runtime + private final ResultPartitionType runtimeType; // -------------------------------------------------------------------------------------------- @@ -46,12 +53,13 @@ public IntermediateDataSet(AbstractJobVertex producer) { } public IntermediateDataSet(IntermediateDataSetID id, AbstractJobVertex producer) { - if (id == null || producer == null) { - throw new NullPointerException(); - } - - this.id = id; - this.producer = producer; + this(id, ResultPartitionType.PIPELINED, producer); + } + + public IntermediateDataSet(IntermediateDataSetID id, ResultPartitionType runtimeType, AbstractJobVertex producer) { + this.id = checkNotNull(id); + this.producer = checkNotNull(producer); + this.runtimeType = checkNotNull(runtimeType); } // -------------------------------------------------------------------------------------------- @@ -67,6 +75,10 @@ public AbstractJobVertex getProducer() { public List getConsumers() { return this.consumers; } + + public ResultPartitionType getRuntimeType() { + return runtimeType; + } // -------------------------------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/ResultPartitionID.java similarity index 79% rename from flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/ResultPartitionID.java index 190845d7b3350..0491b03b35e4f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateResultPartitionID.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/ResultPartitionID.java @@ -21,24 +21,24 @@ import io.netty.buffer.ByteBuf; import org.apache.flink.runtime.AbstractID; -public class IntermediateResultPartitionID extends AbstractID { +public class ResultPartitionID extends AbstractID { private static final long serialVersionUID = 1L; /** * Creates an new random intermediate result partition ID. */ - public IntermediateResultPartitionID() { + public ResultPartitionID() { super(); } - public IntermediateResultPartitionID(long lowerPart, long upperPart) { + public ResultPartitionID(long lowerPart, long upperPart) { super(lowerPart, upperPart); } - public static IntermediateResultPartitionID fromByteBuf(ByteBuf buf) { + public static ResultPartitionID fromByteBuf(ByteBuf buf) { long lower = buf.readLong(); long upper = buf.readLong(); - return new IntermediateResultPartitionID(lower, upper); + return new ResultPartitionID(lower, upper); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java index 14e4ae6abf218..cee9ebbf187fc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java @@ -93,9 +93,9 @@ public class HashPartition extends AbstractPagedInputView implements See // ------------------------------------------ Spilling ---------------------------------------------- - private BlockChannelWriter buildSideChannel; // the channel writer for the build side, if partition is spilled + private BlockChannelWriter buildSideChannel; // the channel writer for the build side, if partition is spilled - protected BlockChannelWriter probeSideChannel; // the channel writer from the probe side, if partition is spilled + protected BlockChannelWriter probeSideChannel; // the channel writer from the probe side, if partition is spilled // ------------------------------------------ Restoring ---------------------------------------------- @@ -219,12 +219,12 @@ public long getProbeSideRecordCount() { return this.probeSideRecordCounter; } - public BlockChannelWriter getBuildSideChannel() { + public BlockChannelWriter getBuildSideChannel() { return this.buildSideChannel; } - public BlockChannelWriter getProbeSideChannel() { + public BlockChannelWriter getProbeSideChannel() { return this.probeSideChannel; } @@ -506,7 +506,7 @@ protected static final class BuildSideBuffer extends AbstractPagedOutputView private final MemorySegmentSource memSource; - private BlockChannelWriter writer; + private BlockChannelWriter writer; private int currentBlockNumber; @@ -553,7 +553,7 @@ int getBlockCount() { return this.currentBlockNumber + 1; } - int spill(BlockChannelWriter writer) throws IOException + int spill(BlockChannelWriter writer) throws IOException { this.writer = writer; final int numSegments = this.targetList.size(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java index e69ef172717b2..c0be6115b299f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/MutableHashTable.java @@ -300,7 +300,7 @@ public class MutableHashTable implements MemorySegmentSource { /** * The reader for the spilled-file of the probe partition that is currently read. */ - private BlockChannelReader currentSpilledProbeSide; + private BlockChannelReader currentSpilledProbeSide; /** * The channel enumerator that is used while processing the current partition to create @@ -802,7 +802,7 @@ protected void buildTableFromSpilledPartition(final HashPartition p) thr segments.add(getNextBuffer()); segments.add(getNextBuffer()); - final BlockChannelReader inReader = this.ioManager.createBlockChannelReader(p.getBuildSideChannel().getChannelID()); + final BlockChannelReader inReader = this.ioManager.createBlockChannelReader(p.getBuildSideChannel().getChannelID()); final ChannelReaderInputView inView = new HeaderlessChannelReaderInputView(inReader, segments, p.getBuildSideBlockCount(), p.getLastSegmentLimit(), false); final ChannelReaderInputViewIterator inIter = new ChannelReaderInputViewIterator(inView, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReOpenableHashPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReOpenableHashPartition.java index 56dcfae704167..84868fff8dbde 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReOpenableHashPartition.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReOpenableHashPartition.java @@ -37,7 +37,7 @@ public class ReOpenableHashPartition extends HashPartition { private FileIOChannel.ID initialBuildSideChannel = null; // path to initial build side contents (only for in-memory partitions) - private BlockChannelWriter initialBuildSideWriter = null; + private BlockChannelWriter initialBuildSideWriter = null; private boolean isRestored = false; // marks a restored partition diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java index 346602484806b..d05bd9adaf624 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java @@ -301,7 +301,7 @@ else if (element == endMarker()) { } // create writer - final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); + final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); registerOpenChannelToBeRemovedAtShudown(writer); final ChannelWriterOutputView output = new ChannelWriterOutputView(writer, this.writeMemory, this.memManager.getPageSize()); @@ -457,7 +457,7 @@ protected ChannelWithBlockCount mergeChannels(List channe // create a new channel writer final FileIOChannel.ID mergedChannelID = this.ioManager.createChannel(); registerChannelToBeRemovedAtShudown(mergedChannelID); - final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(mergedChannelID); + final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(mergedChannelID); registerOpenChannelToBeRemovedAtShudown(writer); final ChannelWriterOutputView output = new ChannelWriterOutputView(writer, writeBuffers, this.memManager.getPageSize()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java index cdd5eb493fe20..e9a5285e1ddde 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java @@ -1301,7 +1301,7 @@ else if (element == EOF_MARKER) { registerChannelToBeRemovedAtShudown(channel); // create writer - final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); + final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); registerOpenChannelToBeRemovedAtShudown(writer); final ChannelWriterOutputView output = new ChannelWriterOutputView(writer, this.writeMemory, this.memManager.getPageSize()); @@ -1479,7 +1479,7 @@ protected final MergeIterator getMergingIterator(final List segsForChannel = inputSegments.get(i); // create a reader. if there are multiple segments for the reader, issue multiple together per I/O request - final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel.getChannel()); + final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel.getChannel()); readerList.add(reader); registerOpenChannelToBeRemovedAtShudown(reader); @@ -1564,7 +1564,7 @@ protected ChannelWithBlockCount mergeChannels(List channe // create a new channel writer final FileIOChannel.ID mergedChannelID = this.ioManager.createChannel(); registerChannelToBeRemovedAtShudown(mergedChannelID); - final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(mergedChannelID); + final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(mergedChannelID); registerOpenChannelToBeRemovedAtShudown(writer); final ChannelWriterOutputView output = new ChannelWriterOutputView(writer, writeBuffers, this.memManager.getPageSize()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 5b6259ac11390..1d8e5fd58f528 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -25,7 +25,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.network.api.reader.BufferReader; import org.apache.flink.runtime.io.network.api.writer.BufferWriter; -import org.apache.flink.runtime.io.network.partition.IntermediateResultPartition; +import org.apache.flink.runtime.io.network.partition.ResultPartition; import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.memorymanager.MemoryManager; @@ -361,7 +361,7 @@ public BufferWriter[] getWriters() { return environment != null ? environment.getAllWriters() : null; } - public IntermediateResultPartition[] getProducedPartitions() { + public ResultPartition[] getProducedPartitions() { return environment != null ? environment.getProducedPartitions() : null; } diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobmanagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobmanagerMessages.scala index fbacbd2593f73..801d81c0bff92 100644 --- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobmanagerMessages.scala +++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/JobmanagerMessages.scala @@ -78,14 +78,14 @@ object JobManagerMessages { *

    * There is a call to this method for each * [[org.apache.flink.runtime.executiongraph.ExecutionVertex]] instance once per produced - * [[org.apache.flink.runtime.io.network.partition.IntermediateResultPartition]] instance, + * [[org.apache.flink.runtime.io.network.partition.ResultPartition]] instance, * either when first producing data (for pipelined executions) or when all data has been produced * (for staged executions). *

    * The [[org.apache.flink.runtime.jobmanager.JobManager]] then can decide when to schedule the * partition consumers of the given session. * - * @see [[org.apache.flink.runtime.io.network.partition.IntermediateResultPartition]] + * @see [[org.apache.flink.runtime.io.network.partition.ResultPartition]] */ case class ScheduleOrUpdateConsumers(jobId: JobID, executionId: ExecutionAttemptID, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java index f4c95c985c47d..0830f8137368b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java @@ -144,8 +144,8 @@ public void testBuildDeploymentDescriptor() { assertEquals(2, producedPartitions.size()); assertEquals(1, consumedPartitions.size()); - assertEquals(10, producedPartitions.get(0).getNumberOfQueues()); - assertEquals(10, producedPartitions.get(1).getNumberOfQueues()); + assertEquals(10, producedPartitions.get(0).getNumberOfSubpartitions()); + assertEquals(10, producedPartitions.get(1).getNumberOfSubpartitions()); assertEquals(10, consumedPartitions.get(0).getPartitions().length); } catch (Exception e) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java index c05fccafcdff9..0462b3f90c34d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java @@ -107,7 +107,7 @@ public void testWriteReadSmallRecords() throws Exception // create the writer output view List memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS); - final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); + final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); final ChannelWriterOutputView outView = new ChannelWriterOutputView(writer, memory, MEMORY_PAGE_SIZE); // write a number of pairs @@ -120,7 +120,7 @@ public void testWriteReadSmallRecords() throws Exception // create the reader input view memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS); - final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel); + final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel); final ChannelReaderInputView inView = new ChannelReaderInputView(reader, memory, outView.getBlockCount(), true); generator.reset(); @@ -151,7 +151,7 @@ public void testWriteAndReadLongRecords() throws Exception // create the writer output view List memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS); - final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); + final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); final ChannelWriterOutputView outView = new ChannelWriterOutputView(writer, memory, MEMORY_PAGE_SIZE); // write a number of pairs @@ -164,7 +164,7 @@ public void testWriteAndReadLongRecords() throws Exception // create the reader input view memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS); - final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel); + final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel); final ChannelReaderInputView inView = new ChannelReaderInputView(reader, memory, outView.getBlockCount(), true); generator.reset(); @@ -192,7 +192,7 @@ public void testReadTooMany() throws Exception // create the writer output view List memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS); - final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); + final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); final ChannelWriterOutputView outView = new ChannelWriterOutputView(writer, memory, MEMORY_PAGE_SIZE); // write a number of pairs @@ -205,7 +205,7 @@ public void testReadTooMany() throws Exception // create the reader input view memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS); - final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel); + final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel); final ChannelReaderInputView inView = new ChannelReaderInputView(reader, memory, outView.getBlockCount(), true); generator.reset(); @@ -243,7 +243,7 @@ public void testReadWithoutKnownBlockCount() throws Exception // create the writer output view List memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS); - final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); + final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); final ChannelWriterOutputView outView = new ChannelWriterOutputView(writer, memory, MEMORY_PAGE_SIZE); // write a number of pairs @@ -256,7 +256,7 @@ public void testReadWithoutKnownBlockCount() throws Exception // create the reader input view memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS); - final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel); + final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel); final ChannelReaderInputView inView = new ChannelReaderInputView(reader, memory, true); generator.reset(); @@ -287,7 +287,7 @@ public void testWriteReadOneBufferOnly() throws Exception // create the writer output view List memory = this.memoryManager.allocatePages(this.parentTask, 1); - final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); + final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); final ChannelWriterOutputView outView = new ChannelWriterOutputView(writer, memory, MEMORY_PAGE_SIZE); // write a number of pairs @@ -300,7 +300,7 @@ public void testWriteReadOneBufferOnly() throws Exception // create the reader input view memory = this.memoryManager.allocatePages(this.parentTask, 1); - final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel); + final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel); final ChannelReaderInputView inView = new ChannelReaderInputView(reader, memory, outView.getBlockCount(), true); generator.reset(); @@ -331,7 +331,7 @@ public void testWriteReadNotAll() throws Exception // create the writer output view List memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS); - final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); + final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channel); final ChannelWriterOutputView outView = new ChannelWriterOutputView(writer, memory, MEMORY_PAGE_SIZE); // write a number of pairs @@ -344,7 +344,7 @@ public void testWriteReadNotAll() throws Exception // create the reader input view memory = this.memoryManager.allocatePages(this.parentTask, NUM_MEMORY_SEGMENTS); - final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel); + final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channel); final ChannelReaderInputView inView = new ChannelReaderInputView(reader, memory, outView.getBlockCount(), true); generator.reset(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java new file mode 100644 index 0000000000000..f33448a0a1595 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriterTest.java @@ -0,0 +1,178 @@ +/* + * 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.disk.iomanager; + +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.partition.MockNotificationListener; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicReference; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; + +public class AsynchronousBufferFileWriterTest { + + private static final IOManager ioManager = new IOManagerAsync(); + + private static final Buffer mockBuffer = mock(Buffer.class); + + private AsynchronousBufferFileWriter writer; + + @Before + public void setUp() throws IOException { + writer = new AsynchronousBufferFileWriter(ioManager.createChannel(), new RequestQueue()); + } + + @Test + public void testAddAndHandleRequest() throws Exception { + addRequest(); + assertEquals("Didn't increment number of outstanding requests.", 1, writer.getNumberOfOutstandingRequests()); + + handleRequest(); + assertEquals("Didn't decrement number of outstanding requests.", 0, writer.getNumberOfOutstandingRequests()); + } + + @Test + public void testSubscribe() throws Exception { + final MockNotificationListener listener = new MockNotificationListener(); + + // Unsuccessful subscription, because no outstanding requests + assertFalse("Allowed to subscribe w/o any outstanding requests.", writer.subscribe(listener)); + + // Successful subscription + addRequest(); + assertTrue("Didn't allow to subscribe.", writer.subscribe(listener)); + + // Test notification + handleRequest(); + + assertEquals("Listener was not notified.", 1, listener.getNumberOfNotifications()); + } + + @Test + public void testSubscribeAndClose() throws IOException, InterruptedException { + final MockNotificationListener listener = new MockNotificationListener(); + + final AtomicReference error = new AtomicReference(); + + final CountDownLatch sync = new CountDownLatch(1); + + addRequest(); + addRequest(); + + writer.subscribe(listener); + + final Thread asyncCloseThread = new Thread(new Runnable() { + @Override + public void run() { + try { + writer.close(); + } + catch (Throwable t) { + error.set(t); + } + finally { + sync.countDown(); + } + } + }); + + asyncCloseThread.start(); + + handleRequest(); + handleRequest(); + + sync.await(); + + assertEquals("Listener was not notified.", 1, listener.getNumberOfNotifications()); + } + + @Test + public void testConcurrentSubscribeAndHandleRequest() throws Exception { + final ExecutorService executor = Executors.newFixedThreadPool(2); + + final MockNotificationListener listener = new MockNotificationListener(); + + final Callable subscriber = new Callable() { + @Override + public Boolean call() throws Exception { + return writer.subscribe(listener); + } + }; + + final Callable requestHandler = new Callable() { + @Override + public Void call() throws Exception { + handleRequest(); + return null; + } + }; + + try { + // Repeat this to provoke races + for (int i = 0; i < 50000; i++) { + listener.reset(); + + addRequest(); + + Future handleRequestFuture = executor.submit(requestHandler); + Future subscribeFuture = executor.submit(subscriber); + + handleRequestFuture.get(); + + try { + if (subscribeFuture.get()) { + assertEquals("Race: Successfully subscribed, but was never notified.", 1, listener.getNumberOfNotifications()); + } + else { + assertEquals("Race: Never subscribed successfully, but was notified.", 0, listener.getNumberOfNotifications()); + } + } + catch (Throwable t) { + System.out.println(i); + Assert.fail(t.getMessage()); + } + } + } + finally { + executor.shutdownNow(); + } + } + + // ------------------------------------------------------------------------ + + private void addRequest() throws IOException { + writer.writeBlock(mockBuffer); + } + + private void handleRequest() { + writer.handleProcessedBuffer(mockBuffer, null); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java new file mode 100644 index 0000000000000..f2bc25cdd8575 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java @@ -0,0 +1,198 @@ +/* + * 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.disk.iomanager; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferRecycler; +import org.apache.flink.runtime.testutils.DiscardingRecycler; +import org.apache.flink.runtime.util.event.NotificationListener; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Random; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.LinkedBlockingQueue; + +import static junit.framework.TestCase.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class BufferFileWriterFileSegmentReaderTest { + + private static final int BUFFER_SIZE = 32 * 1024; + + private static final BufferRecycler BUFFER_RECYCLER = new DiscardingRecycler(); + + private static final Random random = new Random(); + + private static final IOManager ioManager = new IOManagerAsync(); + + private BufferFileWriter writer; + + private AsynchronousBufferFileSegmentReader reader; + + private LinkedBlockingQueue returnedFileSegments = new LinkedBlockingQueue(); + + @Before + public void setUpWriterAndReader() { + final FileIOChannel.ID channel = ioManager.createChannel(); + + try { + writer = ioManager.createBufferFileWriter(channel); + reader = (AsynchronousBufferFileSegmentReader) ioManager.createBufferFileSegmentReader(channel, new QueuingCallback(returnedFileSegments)); + } + catch (IOException e) { + if (writer != null) { + writer.deleteChannel(); + } + + if (reader != null) { + reader.deleteChannel(); + } + + fail("Failed to setup writer and reader."); + } + } + + @After + public void tearDownWriterAndReader() { + if (writer != null) { + writer.deleteChannel(); + } + + if (reader != null) { + reader.deleteChannel(); + } + + returnedFileSegments.clear(); + } + + @Test + public void testWriteRead() throws IOException, InterruptedException { + int numBuffers = 1024; + int currentNumber = 0; + + final int minBufferSize = BUFFER_SIZE / 4; + + // Write buffers filled with ascending numbers... + for (int i = 0; i < numBuffers; i++) { + final Buffer buffer = createBuffer(); + + int size = getNextMultipleOf(getRandomNumberInRange(minBufferSize, BUFFER_SIZE), 4); + + buffer.setSize(size); + + currentNumber = fillBufferWithAscendingNumbers(buffer, currentNumber); + + writer.writeBlock(buffer); + } + + // Make sure that the writes are finished + writer.close(); + + // Read buffers back in... + for (int i = 0; i < numBuffers; i++) { + assertFalse(reader.isConsumed()); + reader.read(); + } + + // Wait for all requests to be finished + final CountDownLatch sync = new CountDownLatch(1); + final NotificationListener listener = new NotificationListener() { + @Override + public void onNotification() { + sync.countDown(); + } + }; + + if (reader.subscribe(listener)) { + sync.await(); + } + + assertTrue(reader.isConsumed()); + + // Verify that the content is the same + assertEquals("Read less buffers than written.", numBuffers, returnedFileSegments.size()); + + currentNumber = 0; + FileSegment fileSegment; + + ByteBuffer buffer = ByteBuffer.allocate(BUFFER_SIZE); + + while ((fileSegment = returnedFileSegments.poll()) != null) { + buffer.position(0); + buffer.limit(fileSegment.getLength()); + + fileSegment.getFileChannel().read(buffer, fileSegment.getPosition()); + + currentNumber = verifyBufferFilledWithAscendingNumbers(new Buffer(new MemorySegment(buffer.array()), BUFFER_RECYCLER), currentNumber, fileSegment.getLength()); + } + + reader.close(); + } + + // ------------------------------------------------------------------------ + + private int getRandomNumberInRange(int min, int max) { + return random.nextInt((max - min) + 1) + min; + } + + private int getNextMultipleOf(int number, int multiple) { + final int mod = number % multiple; + + if (mod == 0) { + return number; + } + + return number + multiple - mod; + } + + private Buffer createBuffer() { + return new Buffer(new MemorySegment(new byte[BUFFER_SIZE]), BUFFER_RECYCLER); + } + + public static int fillBufferWithAscendingNumbers(Buffer buffer, int currentNumber) { + MemorySegment segment = buffer.getMemorySegment(); + + final int size = buffer.getSize(); + + for (int i = 0; i < size; i += 4) { + segment.putInt(i, currentNumber++); + } + + return currentNumber; + } + + private int verifyBufferFilledWithAscendingNumbers(Buffer buffer, int currentNumber, int size) { + MemorySegment segment = buffer.getMemorySegment(); + + for (int i = 0; i < size; i += 4) { + if (segment.getInt(i) != currentNumber++) { + throw new IllegalStateException("Read unexpected number from buffer."); + } + } + + return currentNumber; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java new file mode 100644 index 0000000000000..2150a43263ba4 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java @@ -0,0 +1,225 @@ +/* + * 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.disk.iomanager; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferRecycler; +import org.apache.flink.runtime.testutils.DiscardingRecycler; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Random; +import java.util.concurrent.LinkedBlockingQueue; + +import static junit.framework.TestCase.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class BufferFileWriterReaderTest { + + private static final int BUFFER_SIZE = 32 * 1024; + + private static final BufferRecycler BUFFER_RECYCLER = new DiscardingRecycler(); + + private static final Random random = new Random(); + + private static final IOManager ioManager = new IOManagerAsync(); + + private BufferFileWriter writer; + + private BufferFileReader reader; + + private LinkedBlockingQueue returnedBuffers = new LinkedBlockingQueue(); + + @Before + public void setUpWriterAndReader() { + final FileIOChannel.ID channel = ioManager.createChannel(); + + try { + writer = ioManager.createBufferFileWriter(channel); + reader = ioManager.createBufferFileReader(channel, new QueuingCallback(returnedBuffers)); + } + catch (IOException e) { + if (writer != null) { + writer.deleteChannel(); + } + + if (reader != null) { + reader.deleteChannel(); + } + + fail("Failed to setup writer and reader."); + } + } + + @After + public void tearDownWriterAndReader() { + if (writer != null) { + writer.deleteChannel(); + } + + if (reader != null) { + reader.deleteChannel(); + } + + returnedBuffers.clear(); + } + + @Test + public void testWriteRead() throws IOException { + int numBuffers = 1024; + int currentNumber = 0; + + final int minBufferSize = BUFFER_SIZE / 4; + + // Write buffers filled with ascending numbers... + for (int i = 0; i < numBuffers; i++) { + final Buffer buffer = createBuffer(); + + int size = getNextMultipleOf(getRandomNumberInRange(minBufferSize, BUFFER_SIZE), 4); + + buffer.setSize(size); + + currentNumber = fillBufferWithAscendingNumbers(buffer, currentNumber); + + writer.writeBlock(buffer); + } + + // Make sure that the writes are finished + writer.close(); + + // Read buffers back in... + for (int i = 0; i < numBuffers; i++) { + assertFalse(reader.isConsumed()); + reader.readInto(createBuffer()); + } + + reader.close(); + + assertTrue(reader.isConsumed()); + + // Verify that the content is the same + assertEquals("Read less buffers than written.", numBuffers, returnedBuffers.size()); + + currentNumber = 0; + Buffer buffer; + + while ((buffer = returnedBuffers.poll()) != null) { + currentNumber = verifyBufferFilledWithAscendingNumbers(buffer, currentNumber); + } + } + + @Test + public void testWriteSkipRead() throws IOException { + int numBuffers = 1024; + int currentNumber = 0; + + final int minBufferSize = BUFFER_SIZE / 4; + + // Write buffers filled with ascending numbers... + for (int i = 0; i < numBuffers; i++) { + final Buffer buffer = createBuffer(); + + currentNumber = fillBufferWithAscendingNumbers(buffer, currentNumber); + + writer.writeBlock(buffer); + } + + // Make sure that the writes are finished + writer.close(); + + final int toSkip = 32; + + // Skip first buffers... + reader.seekToPosition((8 + BUFFER_SIZE) * toSkip); + + numBuffers -= toSkip; + + // Read buffers back in... + for (int i = 0; i < numBuffers; i++) { + assertFalse(reader.isConsumed()); + reader.readInto(createBuffer()); + } + + reader.close(); + + assertTrue(reader.isConsumed()); + + // Verify that the content is the same + assertEquals("Read less buffers than written.", numBuffers, returnedBuffers.size()); + + // Start number after skipped buffers... + currentNumber = (BUFFER_SIZE / 4) * toSkip; + + Buffer buffer; + while ((buffer = returnedBuffers.poll()) != null) { + currentNumber = verifyBufferFilledWithAscendingNumbers(buffer, currentNumber); + } + } + + // ------------------------------------------------------------------------ + + private int getRandomNumberInRange(int min, int max) { + return random.nextInt((max - min) + 1) + min; + } + + private int getNextMultipleOf(int number, int multiple) { + final int mod = number % multiple; + + if (mod == 0) { + return number; + } + + return number + multiple - mod; + } + + private Buffer createBuffer() { + return new Buffer(new MemorySegment(new byte[BUFFER_SIZE]), BUFFER_RECYCLER); + } + + public static int fillBufferWithAscendingNumbers(Buffer buffer, int currentNumber) { + MemorySegment segment = buffer.getMemorySegment(); + + final int size = buffer.getSize(); + + for (int i = 0; i < size; i += 4) { + segment.putInt(i, currentNumber++); + } + + return currentNumber; + } + + private int verifyBufferFilledWithAscendingNumbers(Buffer buffer, int currentNumber) { + MemorySegment segment = buffer.getMemorySegment(); + + final int size = buffer.getSize(); + + for (int i = 0; i < size; i += 4) { + if (segment.getInt(i) != currentNumber++) { + throw new IllegalStateException("Read unexpected number from buffer."); + } + } + + return currentNumber; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java index 297eeed42d09f..435588fb13cd8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java @@ -18,8 +18,7 @@ package org.apache.flink.runtime.io.disk.iomanager; -import static org.junit.Assert.*; - +import org.apache.flink.core.memory.MemorySegment; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -29,12 +28,8 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.flink.core.memory.MemorySegment; -import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader; -import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter; -import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel; -import org.apache.flink.runtime.io.disk.iomanager.ReadRequest; -import org.apache.flink.runtime.io.disk.iomanager.WriteRequest; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; public class IOManagerAsyncTest { @@ -65,7 +60,7 @@ public void channelReadWriteOneSegment() { try { final FileIOChannel.ID channelID = this.ioManager.createChannel(); - final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channelID); + final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channelID); MemorySegment memSeg = new MemorySegment(new byte[32 * 1024]); @@ -75,15 +70,15 @@ public void channelReadWriteOneSegment() { } writer.writeBlock(memSeg); - memSeg = writer.getNextReturnedSegment(); + memSeg = writer.getNextReturnedBlock(); } writer.close(); - final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channelID); + final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channelID); for (int i = 0; i < NUM_IOS; i++) { reader.readBlock(memSeg); - memSeg = reader.getNextReturnedSegment(); + memSeg = reader.getNextReturnedBlock(); for (int pos = 0; pos < memSeg.size(); pos += 4) { if (memSeg.getInt(pos) != i) { @@ -112,10 +107,10 @@ public void channelReadWriteMultipleSegments() { } final FileIOChannel.ID channelID = this.ioManager.createChannel(); - final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channelID); + final BlockChannelWriter writer = this.ioManager.createBlockChannelWriter(channelID); for (int i = 0; i < NUM_IOS; i++) { - final MemorySegment memSeg = memSegs.isEmpty() ? writer.getNextReturnedSegment() : memSegs.remove(memSegs.size() - 1); + final MemorySegment memSeg = memSegs.isEmpty() ? writer.getNextReturnedBlock() : memSegs.remove(memSegs.size() - 1); for (int pos = 0; pos < memSeg.size(); pos += 4) { memSeg.putInt(pos, i); @@ -127,16 +122,16 @@ public void channelReadWriteMultipleSegments() { // get back the memory while (memSegs.size() < NUM_SEGS) { - memSegs.add(writer.getNextReturnedSegment()); + memSegs.add(writer.getNextReturnedBlock()); } - final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channelID); + final BlockChannelReader reader = this.ioManager.createBlockChannelReader(channelID); while(!memSegs.isEmpty()) { reader.readBlock(memSegs.remove(0)); } for (int i = 0; i < NUM_IOS; i++) { - final MemorySegment memSeg = reader.getNextReturnedSegment(); + final MemorySegment memSeg = reader.getNextReturnedBlock(); for (int pos = 0; pos < memSeg.size(); pos += 4) { if (memSeg.getInt(pos) != i) { @@ -150,7 +145,7 @@ public void channelReadWriteMultipleSegments() { // get back the memory while (memSegs.size() < NUM_SEGS) { - memSegs.add(reader.getNextReturnedSegment()); + memSegs.add(reader.getNextReturnedBlock()); } } catch (Exception ex) { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java index f1d53377914fa..d72227c993a80 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java @@ -84,8 +84,8 @@ public void parallelChannelsTest() throws Exception { final AbstractInvokable memOwner = new DefaultMemoryManagerTest.DummyInvokable(); FileIOChannel.ID[] ids = new FileIOChannel.ID[NUM_CHANNELS]; - BlockChannelWriter[] writers = new BlockChannelWriter[NUM_CHANNELS]; - BlockChannelReader[] readers = new BlockChannelReader[NUM_CHANNELS]; + BlockChannelWriter[] writers = new BlockChannelWriter[NUM_CHANNELS]; + BlockChannelReader[] readers = new BlockChannelReader[NUM_CHANNELS]; ChannelWriterOutputView[] outs = new ChannelWriterOutputView[NUM_CHANNELS]; ChannelReaderInputView[] ins = new ChannelReaderInputView[NUM_CHANNELS]; @@ -126,7 +126,7 @@ public void parallelChannelsTest() throws Exception { List memSegs = this.memoryManager.allocatePages(memOwner, rnd.nextInt(MAXIMUM_NUMBER_OF_SEGMENTS_PER_CHANNEL - 1) + 1); - final BlockChannelReader reader = this.ioManager.createBlockChannelReader(ids[i]); + final BlockChannelReader reader = this.ioManager.createBlockChannelReader(ids[i]); final ChannelReaderInputView in = new ChannelReaderInputView(reader, memSegs, false); int nextVal = 0; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java index 9c129e6c256b2..89cc50d682c48 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java @@ -112,8 +112,8 @@ private void testChannelWithSegments(int numSegments) throws Exception final List memory = this.memManager.allocatePages(memoryOwner, numSegments); final FileIOChannel.ID channel = this.ioManager.createChannel(); - BlockChannelWriter writer = null; - BlockChannelReader reader = null; + BlockChannelWriter writer = null; + BlockChannelReader reader = null; try { writer = this.ioManager.createBlockChannelWriter(channel); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java index 4be667a0e8ea2..8186f4fdd437a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerTest.java @@ -20,9 +20,11 @@ import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel.ID; +import org.apache.flink.runtime.io.network.buffer.Buffer; import org.junit.Test; import java.io.File; +import java.io.IOException; import java.util.List; import java.util.concurrent.LinkedBlockingQueue; @@ -89,17 +91,32 @@ public boolean isProperlyShutDown() { } @Override - public BlockChannelWriter createBlockChannelWriter(ID channelID, LinkedBlockingQueue returnQueue) { + public BlockChannelWriter createBlockChannelWriter(ID channelID, LinkedBlockingQueue returnQueue) { throw new UnsupportedOperationException(); } @Override - public BlockChannelWriterWithCallback createBlockChannelWriter(ID channelID, RequestDoneCallback callback) { + public BlockChannelWriterWithCallback createBlockChannelWriter(ID channelID, RequestDoneCallback callback) { throw new UnsupportedOperationException(); } @Override - public BlockChannelReader createBlockChannelReader(ID channelID, LinkedBlockingQueue returnQueue) { + public BlockChannelReader createBlockChannelReader(ID channelID, LinkedBlockingQueue returnQueue) { + throw new UnsupportedOperationException(); + } + + @Override + public BufferFileWriter createBufferFileWriter(ID channelID) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public BufferFileReader createBufferFileReader(ID channelID, RequestDoneCallback callback) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public BufferFileSegmentReader createBufferFileSegmentReader(ID channelID, RequestDoneCallback callback) throws IOException { throw new UnsupportedOperationException(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/MockBufferReader.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/MockBufferReader.java index d439eeb6da355..5fc2c212940af 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/MockBufferReader.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/reader/MockBufferReader.java @@ -29,7 +29,7 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.partition.consumer.InputChannel; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.mockito.stubbing.OngoingStubbing; @@ -50,7 +50,7 @@ public class MockBufferReader { public MockBufferReader() throws IOException { reader = new BufferReader(mock(RuntimeEnvironment.class), MockNetworkEnvironment.getMock(), new IntermediateDataSetID(), 1, 0); - reader.setInputChannel(new IntermediateResultPartitionID(), inputChannel); + reader.setInputChannel(new ResultPartitionID(), inputChannel); } MockBufferReader read(Buffer buffer) throws IOException { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java index cf6eb9a913520..28862e8bbdb3a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java @@ -124,7 +124,7 @@ public void testCreateDestroy() throws IOException { assertEquals(networkBufferPool.getTotalNumberOfMemorySegments() / 2, second.getNumBuffers()); - first.destroy(); + first.lazyDestroy(); assertEquals(networkBufferPool.getTotalNumberOfMemorySegments(), second.getNumBuffers()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java index 17a079c1b39eb..49f26a58f1ed7 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java @@ -77,7 +77,7 @@ public void testExceptionAfterRecycle() throws Throwable { // declared methods as Mockito adds some of its own. for (final Method method : buffer.getClass().getDeclaredMethods()) { if (Modifier.isPublic(method.getModifiers()) && !method.getName().equals("toString") - && !method.getName().equals("isRecycled") && !method.getName().equals("isBuffer")) { + && !method.getName().equals("isRecycled") && !method.getName().equals("isBuffer") && !method.getName().equals("getSize")) { // Get method of the spied buffer to allow argument matchers final Method spyMethod = spyBuffer.getClass().getDeclaredMethod(method.getName(), method.getParameterTypes()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java index 8609db00346d6..5636eea985233 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java @@ -63,7 +63,7 @@ public void setupLocalBufferPool() { @After public void destroyAndVerifyAllBuffersReturned() throws IOException { if (!localBufferPool.isDestroyed()) { - localBufferPool.destroy(); + localBufferPool.lazyDestroy(); } String msg = "Did not return all buffers to memory segment pool after test."; @@ -99,7 +99,7 @@ public void testRequestMoreThanAvailable() throws IOException { @Test public void testRequestAfterDestroy() throws IOException { - localBufferPool.destroy(); + localBufferPool.lazyDestroy(); assertNull(localBufferPool.requestBuffer()); } @@ -114,7 +114,7 @@ public void testRecycleAfterDestroy() throws IOException { requests.add(localBufferPool.requestBuffer()); } - localBufferPool.destroy(); + localBufferPool.lazyDestroy(); // All buffers have been requested, but can not be returned yet. assertEquals(numBuffers, getNumRequestedFromMemorySegmentPool()); @@ -224,7 +224,7 @@ public void testCancelPendingRequestsAfterDestroy() throws IOException { localBufferPool.addListener(listener); - localBufferPool.destroy(); + localBufferPool.lazyDestroy(); available.recycle(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java index 991af13ddb9e9..7ee6ca8f8b4e6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java @@ -26,7 +26,7 @@ import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferRecycler; import org.apache.flink.runtime.io.network.partition.consumer.InputChannelID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; import org.junit.Test; import java.nio.ByteBuffer; @@ -123,7 +123,7 @@ public void testEncodeDecode() { } { - NettyMessage.PartitionRequest expected = new NettyMessage.PartitionRequest(new ExecutionAttemptID(), new IntermediateResultPartitionID(), random.nextInt(), new InputChannelID()); + NettyMessage.PartitionRequest expected = new NettyMessage.PartitionRequest(new ExecutionAttemptID(), new ResultPartitionID(), random.nextInt(), new InputChannelID()); NettyMessage.PartitionRequest actual = encodeAndDecode(expected); assertEquals(expected.producerExecutionId, actual.producerExecutionId); @@ -133,7 +133,7 @@ public void testEncodeDecode() { } { - NettyMessage.TaskEventRequest expected = new NettyMessage.TaskEventRequest(new IntegerTaskEvent(random.nextInt()), new ExecutionAttemptID(), new IntermediateResultPartitionID(), new InputChannelID()); + NettyMessage.TaskEventRequest expected = new NettyMessage.TaskEventRequest(new IntegerTaskEvent(random.nextInt()), new ExecutionAttemptID(), new ResultPartitionID(), new InputChannelID()); NettyMessage.TaskEventRequest actual = encodeAndDecode(expected); assertEquals(expected.executionId, actual.executionId); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockBufferFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockBufferFactory.java new file mode 100644 index 0000000000000..8ca38e328a708 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockBufferFactory.java @@ -0,0 +1,87 @@ +/* + * 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; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferRecycler; +import org.apache.flink.runtime.testutils.DiscardingRecycler; + +public class MockBufferFactory { + + private static final int BUFFER_SIZE = 32 * 1024; + + private static final BufferRecycler BUFFER_RECYCLER = new DiscardingRecycler(); + + private int currentCreateNumber; + + private int currentVerifyNumber; + + public Buffer create() { + final MemorySegment segment = new MemorySegment(new byte[BUFFER_SIZE]); + + for (int i = 0; i < BUFFER_SIZE; i += 4) { + segment.putInt(i, currentCreateNumber++); + } + + return new Buffer(segment, BUFFER_RECYCLER); + } + + public boolean verify(Buffer buffer) { + final MemorySegment segment = buffer.getMemorySegment(); + + for (int i = 0; i < BUFFER_SIZE; i += 4) { + if (segment.getInt(i) != currentVerifyNumber++) { + return false; + } + } + + return true; + } + + public static Buffer createBuffer() { + return new Buffer(new MemorySegment(new byte[BUFFER_SIZE]), BUFFER_RECYCLER); + } + + public static int fillBufferWithAscendingNumbers(Buffer buffer, int currentNumber) { + MemorySegment segment = buffer.getMemorySegment(); + + final int size = buffer.getSize(); + + for (int i = 0; i < size; i += 4) { + segment.putInt(i, currentNumber++); + } + + return currentNumber; + } + + public static int verifyBufferFilledWithAscendingNumbers(Buffer buffer, int currentNumber) { + MemorySegment segment = buffer.getMemorySegment(); + + final int size = buffer.getSize(); + + for (int i = 0; i < size; i += 4) { + if (segment.getInt(i) != currentNumber++) { + throw new IllegalStateException("Read unexpected number from buffer: was " + segment.getInt(i) + ", expected " + (currentNumber - 1)); + } + } + + return currentNumber; + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockBufferProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockBufferProvider.java new file mode 100644 index 0000000000000..7d1d948cc77af --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockBufferProvider.java @@ -0,0 +1,117 @@ +/* + * 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; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferProvider; +import org.apache.flink.runtime.io.network.buffer.BufferRecycler; +import org.apache.flink.runtime.util.event.EventListener; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.Queue; + +import static com.google.common.base.Preconditions.checkArgument; + +/** + * A mock buffer provider. + */ +public class MockBufferProvider implements BufferProvider, BufferRecycler { + + private final int numBuffers; + + private final int bufferSize; + + private final Queue buffers; + + public MockBufferProvider(int bufferSize) { + this.bufferSize = bufferSize; + + this.numBuffers = -1; // infinite number of buffers + this.buffers = null; // don't pool buffers + } + + public MockBufferProvider(int bufferSize, int numBuffers) { + checkArgument(numBuffers > 0); + checkArgument(bufferSize > 0); + + this.numBuffers = numBuffers; + this.bufferSize = bufferSize; + + this.buffers = new ArrayDeque(numBuffers); + for (int i = 0; i < numBuffers; i++) { + buffers.add(new Buffer(new MemorySegment(new byte[bufferSize]), this)); + } + } + + @Override + public Buffer requestBuffer() throws IOException { + if (numBuffers > 0) { + synchronized (buffers) { + return buffers.poll(); + } + } + + return new Buffer(new MemorySegment(new byte[bufferSize]), this); + } + + @Override + public Buffer requestBufferBlocking() throws IOException, InterruptedException { + if (numBuffers > 0) { + synchronized (buffers) { + Buffer buffer = buffers.poll(); + + if (buffer != null) { + return buffer; + } + + while (buffer == null) { + buffers.wait(1000); + buffer = buffers.poll(); + } + + return buffer; + } + } + + return new Buffer(new MemorySegment(new byte[bufferSize]), this); + } + + @Override + public boolean addListener(EventListener listener) { + throw new UnsupportedOperationException("addListener() not supported by mock buffer provider."); + } + + @Override + public void recycle(MemorySegment memorySegment) { + if (numBuffers > 0) { + synchronized (buffers) { + buffers.add(new Buffer(memorySegment, this)); + buffers.notify(); + } + } + } + + // -------------------------------------------------------------------- + + public boolean verifyAllBuffersAvailable() { + return numBuffers == buffers.size(); + } +} \ No newline at end of file diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockConsumer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockConsumer.java index 79494aa033abf..dc5028a114d50 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockConsumer.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockConsumer.java @@ -20,7 +20,6 @@ import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.io.network.buffer.Buffer; -import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicReference; @@ -29,13 +28,13 @@ public class MockConsumer implements Callable { private static final int SLEEP_TIME_MS = 20; - private final IntermediateResultPartitionQueueIterator iterator; + private final ResultSubpartitionView iterator; private final boolean slowConsumer; private final AtomicReference error = new AtomicReference(); - public MockConsumer(IntermediateResultPartitionQueueIterator iterator, boolean slowConsumer) { + public MockConsumer(ResultSubpartitionView iterator, boolean slowConsumer) { this.iterator = iterator; this.slowConsumer = slowConsumer; } @@ -89,7 +88,7 @@ public Throwable getError() { private int verifyBufferFilledWithAscendingNumbers(Buffer buffer, int currentNumber) { MemorySegment segment = buffer.getMemorySegment(); - for (int i = 4; i < segment.size(); i += 4) { + for (int i = 0; i < segment.size(); i += 4) { if (segment.getInt(i) != currentNumber++) { throw new IllegalStateException("Read unexpected number from buffer."); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockNotificationListener.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockNotificationListener.java index 928ac51c7529f..3f4491660e1c9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockNotificationListener.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockNotificationListener.java @@ -35,8 +35,15 @@ public void onNotification() { } } - public void waitForNotification() throws InterruptedException { + public void waitForNotification(int current) throws InterruptedException { + synchronized (numNotifications) { + while (current == numNotifications.get()) { + numNotifications.wait(); + } + } + } + public void waitForNotification() throws InterruptedException { int current = numNotifications.get(); synchronized (numNotifications) { @@ -49,4 +56,8 @@ public void waitForNotification() throws InterruptedException { public int getNumberOfNotifications() { return numNotifications.get(); } + + public void reset() { + numNotifications.set(0); + } } \ No newline at end of file diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockProducer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockProducer.java index 1cfe75d65eafc..473e3e0f48ddb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockProducer.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/MockProducer.java @@ -21,7 +21,6 @@ import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferPool; -import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueue; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicInteger; @@ -29,9 +28,9 @@ public class MockProducer implements Callable { - private static final int SLEEP_TIME_MS = 20; + public static final int SLEEP_TIME_MS = 20; - private final IntermediateResultPartitionQueue queue; + private final ResultSubpartition queue; private final BufferPool bufferPool; @@ -43,7 +42,7 @@ public class MockProducer implements Callable { private final AtomicReference error = new AtomicReference(); - public MockProducer(IntermediateResultPartitionQueue queue, BufferPool bufferPool, int numBuffersToProduce, boolean slowProducer) { + public MockProducer(ResultSubpartition queue, BufferPool bufferPool, int numBuffersToProduce, boolean slowProducer) { this.queue = queue; this.bufferPool = bufferPool; this.numBuffersToProduce = numBuffersToProduce; @@ -57,7 +56,7 @@ public Boolean call() throws Exception { for (int i = 0; i < numBuffersToProduce; i++) { if (i >= discardAfter.get()) { - queue.discard(); + queue.release(); return true; } @@ -97,7 +96,7 @@ public Throwable getError() { public static int fillBufferWithAscendingNumbers(Buffer buffer, int currentNumber) { MemorySegment segment = buffer.getMemorySegment(); - for (int i = 4; i < segment.size(); i += 4) { + for (int i = 0; i < segment.size(); i += 4) { segment.putInt(i, currentNumber++); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/queue/PipelinedPartitionQueueTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java similarity index 82% rename from flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/queue/PipelinedPartitionQueueTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java index cff03ac9426a4..cff0b06aac23d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/queue/PipelinedPartitionQueueTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PipelinedSubpartitionTest.java @@ -16,17 +16,14 @@ * limitations under the License. */ -package org.apache.flink.runtime.io.network.partition.queue; +package org.apache.flink.runtime.io.network.partition; import com.google.common.base.Optional; import org.apache.flink.runtime.io.network.buffer.Buffer; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.BufferProvider; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; -import org.apache.flink.runtime.io.network.partition.MockConsumer; -import org.apache.flink.runtime.io.network.partition.MockNotificationListener; -import org.apache.flink.runtime.io.network.partition.MockProducer; -import org.apache.flink.runtime.io.network.partition.queue.IntermediateResultPartitionQueueIterator.AlreadySubscribedException; +import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView.AlreadySubscribedException; import org.apache.flink.runtime.util.event.NotificationListener; import org.junit.Before; import org.junit.Test; @@ -48,7 +45,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -public class PipelinedPartitionQueueTest { +public class PipelinedSubpartitionTest { private static final int NUM_BUFFERS = 1024; @@ -56,24 +53,24 @@ public class PipelinedPartitionQueueTest { private static final NetworkBufferPool networkBuffers = new NetworkBufferPool(NUM_BUFFERS, BUFFER_SIZE); - private PipelinedPartitionQueue queue; + private PipelinedSubpartition queue; @Before public void setup() { - this.queue = new PipelinedPartitionQueue(); + this.queue = new PipelinedSubpartition(0, mock(ResultPartition.class)); } - @Test(expected = IllegalQueueIteratorRequestException.class) + @Test(expected = IllegalSubpartitionRequestException.class) public void testExceptionWhenMultipleConsumers() throws IOException { - queue.getQueueIterator(Optional.absent()); + queue.getReadView(Optional.absent()); // This queue is only consumable once, so this should throw an Exception - queue.getQueueIterator(Optional.absent()); + queue.getReadView(Optional.absent()); } @Test(expected = AlreadySubscribedException.class) public void testExceptionWhenMultipleSubscribers() throws IOException { - IntermediateResultPartitionQueueIterator iterator = queue.getQueueIterator(Optional.absent()); + ResultSubpartitionView iterator = queue.getReadView(Optional.absent()); NotificationListener listener = mock(NotificationListener.class); @@ -90,7 +87,7 @@ public void testProduceConsume() throws Exception { MockNotificationListener listener = new MockNotificationListener(); - IntermediateResultPartitionQueueIterator iterator = queue.getQueueIterator(Optional.absent()); + ResultSubpartitionView iterator = queue.getReadView(Optional.absent()); // Empty queue => should return null assertNull(iterator.getNextBuffer()); @@ -124,13 +121,13 @@ public void testProduceConsume() throws Exception { @Test public void testDiscardingProduceWhileSubscribedConsumer() throws IOException { - IntermediateResultPartitionQueueIterator iterator = queue.getQueueIterator(Optional.absent()); + ResultSubpartitionView iterator = queue.getReadView(Optional.absent()); NotificationListener listener = mock(NotificationListener.class); assertTrue(iterator.subscribe(listener)); - queue.discard(); + queue.release(); verify(listener, times(1)).onNotification(); @@ -185,7 +182,7 @@ private void doTestConcurrentProduceConsume(boolean slowProducer, boolean slowCo producer.discardAfter(new Random().nextInt(numBuffersToProduce)); } - MockConsumer consumer = new MockConsumer(queue.getQueueIterator(Optional.absent()), slowConsumer); + MockConsumer consumer = new MockConsumer(queue.getReadView(Optional.absent()), slowConsumer); ExecutorService executorService = Executors.newCachedThreadPool(); @@ -195,8 +192,8 @@ private void doTestConcurrentProduceConsume(boolean slowProducer, boolean slowCo boolean success = false; try { - success = producerSuccess.get(5, TimeUnit.SECONDS); - success &= consumerSuccess.get(5, TimeUnit.SECONDS); + success = producerSuccess.get(60, TimeUnit.SECONDS); + success &= consumerSuccess.get(60, TimeUnit.SECONDS); } catch (Throwable t) { t.printStackTrace(); @@ -214,7 +211,7 @@ private void doTestConcurrentProduceConsume(boolean slowProducer, boolean slowCo fail("Unexpected failure during test: " + t.getMessage() + ". Producer error: " + producer.getError() + ", consumer error: " + consumer.getError()); } - producerBufferPool.destroy(); + producerBufferPool.lazyDestroy(); assertTrue(success); } finally { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillablePartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillablePartitionTest.java new file mode 100644 index 0000000000000..ea39abffb82f0 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillablePartitionTest.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; + +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.io.network.buffer.BufferPool; +import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; +import org.apache.flink.runtime.io.network.partition.MockProducer; +import org.apache.flink.runtime.io.network.partition.ResultPartition; +import org.apache.flink.runtime.io.network.partition.SpillableSubpartition; +import org.junit.Before; +import org.junit.Test; + +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; + +public class SpillablePartitionTest { + + private static final IOManager ioManager = new IOManagerAsync(); + + private static final int NUM_BUFFERS = 1024; + + private static final int BUFFER_SIZE = 32 * 1024; + + private static final NetworkBufferPool networkBuffers = new NetworkBufferPool(NUM_BUFFERS, BUFFER_SIZE); + + private SpillableSubpartition queue; + + @Before + public void setUp() { + this.queue = new SpillableSubpartition(0, mock(ResultPartition.class), ioManager); + } + + @Test + public void testConcurrentProduceAndSpill() throws Exception { + final ExecutorService executor = Executors.newFixedThreadPool(2); + + final AtomicReference error = new AtomicReference(); + + final int producerNumBuffersToProduce = 64; + final BufferPool producerBufferPool = networkBuffers.createBufferPool(NUM_BUFFERS, true); + final MockProducer producer = new MockProducer(queue, producerBufferPool, producerNumBuffersToProduce, true); + boolean producerSuccess = false; + + try { + final TimerTask memoryThief = new TimerTask() { + @Override + public void run() { + try { + queue.releaseMemory(); + } + catch (Throwable t) { + error.set(t); + } + } + }; + + // Delay the thief to have producer in produce phase + int delay = MockProducer.SLEEP_TIME_MS * producerNumBuffersToProduce / 4; + + new Timer().schedule(memoryThief, delay); + + producerSuccess = executor.submit(producer).get(60, TimeUnit.SECONDS); + } + finally { + executor.shutdownNow(); + + if (!producerSuccess) { + Throwable t = error.get(); + if (t != null && t.getMessage() != null) { + fail("Error during produce: " + t.getMessage()); + } + + fail("Error during produce (but no Exception thrown)."); + } + + assertFalse("Supposed to test spilling behaviour, but queue is still in memory.", queue.isInMemory()); + + producerBufferPool.lazyDestroy(); + + assertEquals("Resource leak: did not return all buffers to network buffer pool.", NUM_BUFFERS, networkBuffers.getNumberOfAvailableMemorySegments()); + } + } + + @Test + public void testInMemoryProduceConsume () { + + } + + @Test + public void testSpilledProduceConsume() throws Exception { + + } + + @Test + public void testSpillingProduceConsume() throws Exception { + + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledPartitionViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledPartitionViewTest.java new file mode 100644 index 0000000000000..aa433b5cba1a8 --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpilledPartitionViewTest.java @@ -0,0 +1,243 @@ +/* + * 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; + +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter; +import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferProvider; +import org.apache.flink.runtime.io.network.buffer.BufferRecycler; +import org.apache.flink.runtime.testutils.DiscardingRecycler; +import org.apache.flink.runtime.util.event.EventListener; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.Queue; +import java.util.Random; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; + +public class SpilledPartitionViewTest { + + private static final int BUFFER_SIZE = 32 * 1024; + + private static final BufferRecycler BUFFER_RECYCLER = new DiscardingRecycler(); + + private static final Random random = new Random(); + + private static final IOManager ioManager = new IOManagerAsync(); + + private FileIOChannel.ID channel; + + private BufferFileWriter writer; + + @Before + public void setUpWriterAndReader() { + channel = ioManager.createChannel(); + + try { + writer = ioManager.createBufferFileWriter(channel); + } + catch (IOException e) { + if (writer != null) { + writer.deleteChannel(); + } + + fail("Failed to setup writer and reader."); + } + } + + @After + public void tearDownWriterAndReader() { + if (writer != null) { + writer.deleteChannel(); + } + } + + @Test + public void testWriteConsume() throws IOException, InterruptedException { + final int numBuffers = 1024; + + int currentNumber = 0; + + final int minBufferSize = BUFFER_SIZE / 4; + + // Write buffers filled with ascending numbers... + for (int i = 0; i < numBuffers; i++) { + final Buffer buffer = createBuffer(); + + int size = getNextMultipleOf(getRandomNumberInRange(minBufferSize, BUFFER_SIZE), 4); + + buffer.setSize(size); + + currentNumber = fillBufferWithAscendingNumbers(buffer, currentNumber); + + writer.writeBlock(buffer); + } + + // Make sure that the writes are finished + writer.close(); + + // - CONSUME ---------------------------------------------------------- + + MockBufferProvider bufferProvider = new MockBufferProvider(1, BUFFER_SIZE); + + SpilledSubpartitionView iterator = new SpilledSubpartitionView(mock(ResultSubpartition.class), ioManager, channel, bufferProvider); + + MockNotificationListener listener = new MockNotificationListener(); + + currentNumber = 0; + int numReadBuffers = 0; + + // Consume + while (true) { + Buffer buffer = iterator.getNextBuffer(); + + if (buffer == null) { + + int current = listener.getNumberOfNotifications(); + + if (iterator.subscribe(listener)) { + listener.waitForNotification(current); + } + else if (iterator.isConsumed()) { + break; + } + } + else { + numReadBuffers++; + + try { + if (buffer.isBuffer()) { + currentNumber = verifyBufferFilledWithAscendingNumbers(buffer, currentNumber); + } + } + finally { + buffer.recycle(); + } + } + } + + assertEquals(numBuffers, numReadBuffers); + } + + // ------------------------------------------------------------------------ + + private int getRandomNumberInRange(int min, int max) { + return random.nextInt((max - min) + 1) + min; + } + + private int getNextMultipleOf(int number, int multiple) { + final int mod = number % multiple; + + if (mod == 0) { + return number; + } + + return number + multiple - mod; + } + + private Buffer createBuffer() { + return new Buffer(new MemorySegment(new byte[BUFFER_SIZE]), BUFFER_RECYCLER); + } + + public static int fillBufferWithAscendingNumbers(Buffer buffer, int currentNumber) { + MemorySegment segment = buffer.getMemorySegment(); + + final int size = buffer.getSize(); + + for (int i = 0; i < size; i += 4) { + segment.putInt(i, currentNumber++); + } + + return currentNumber; + } + + private int verifyBufferFilledWithAscendingNumbers(Buffer buffer, int currentNumber) { + MemorySegment segment = buffer.getMemorySegment(); + + final int size = buffer.getSize(); + + for (int i = 0; i < size; i += 4) { + if (segment.getInt(i) != currentNumber++) { + throw new IllegalStateException("Read unexpected number from buffer: was " + segment.getInt(i) + ", expected " + (currentNumber-1)); + } + } + + return currentNumber; + } + + private static class MockBufferProvider implements BufferProvider, BufferRecycler { + + private final int numBuffers; + + private final int bufferSize; + + private final Queue buffers; + + public MockBufferProvider(int numBuffers, int bufferSize) { + checkArgument(numBuffers > 0); + checkArgument(bufferSize > 0); + + this.numBuffers = numBuffers; + this.bufferSize = bufferSize; + + this.buffers = new ArrayDeque(numBuffers); + for (int i = 0; i < numBuffers; i++) { + buffers.add(new Buffer(new MemorySegment(new byte[bufferSize]), this)); + } + } + + @Override + public Buffer requestBuffer() throws IOException { + return buffers.poll(); + } + + @Override + public Buffer requestBufferBlocking() throws IOException, InterruptedException { + throw new UnsupportedOperationException("requestBufferBlocking() not supported by mock buffer provider."); + } + + @Override + public boolean addListener(EventListener listener) { + throw new UnsupportedOperationException("addListener() not supported by mock buffer provider."); + } + + @Override + public void recycle(MemorySegment memorySegment) { + buffers.add(new Buffer(memorySegment, this)); + } + + // -------------------------------------------------------------------- + + public boolean verifyAllBuffersAvailable() { + return numBuffers == buffers.size(); + } + } + +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java index 60ede476a20e5..1bea4b70b2fe2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java @@ -38,8 +38,8 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID; -import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionType; +import org.apache.flink.runtime.jobgraph.ResultPartitionID; +import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.JobID; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; @@ -308,10 +308,10 @@ public void testRunJobWithForwardChannel() { ActorRef jm = system.actorOf(Props.create(new SimpleLookupJobManagerCreator())); final ActorRef tm = createTaskManager(jm); - IntermediateResultPartitionID partitionId = new IntermediateResultPartitionID(); + ResultPartitionID partitionId = new ResultPartitionID(); List irpdd = new ArrayList(); - irpdd.add(new PartitionDeploymentDescriptor(new IntermediateDataSetID(), partitionId, IntermediateResultPartitionType.PIPELINED, 1)); + irpdd.add(new PartitionDeploymentDescriptor(new IntermediateDataSetID(), partitionId, ResultPartitionType.PIPELINED, 1)); PartitionConsumerDeploymentDescriptor ircdd = new PartitionConsumerDeploymentDescriptor( @@ -400,10 +400,10 @@ public void testCancellingDependentAndStateUpdateFails() { ActorRef jm = system.actorOf(Props.create(new SimpleLookupFailingUpdateJobManagerCreator())); final ActorRef tm = createTaskManager(jm); - IntermediateResultPartitionID partitionId = new IntermediateResultPartitionID(); + ResultPartitionID partitionId = new ResultPartitionID(); List irpdd = new ArrayList(); - irpdd.add(new PartitionDeploymentDescriptor(new IntermediateDataSetID(), partitionId, IntermediateResultPartitionType.PIPELINED, 1)); + irpdd.add(new PartitionDeploymentDescriptor(new IntermediateDataSetID(), partitionId, ResultPartitionType.PIPELINED, 1)); PartitionConsumerDeploymentDescriptor ircdd = new PartitionConsumerDeploymentDescriptor( diff --git a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java index bd4e63a73a1da..b88eb4ebf54d5 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/exampleJavaPrograms/WordCountITCase.java @@ -28,9 +28,9 @@ public class WordCountITCase extends JavaProgramTestBase { protected String resultPath; public WordCountITCase(){ -// setDegreeOfParallelism(4); -// setNumTaskManagers(2); -// setTaskManagerNumSlots(2); + setDegreeOfParallelism(4); + setNumTaskManagers(2); + setTaskManagerNumSlots(2); } @Override From fa42c95a26aa6a025034f77c34ae2bddcf83fda4 Mon Sep 17 00:00:00 2001 From: Ufuk Celebi Date: Mon, 2 Feb 2015 07:50:52 +0100 Subject: [PATCH 2/2] [Distributed runtime] Update Netty dependency to 4.0.25.Final --- flink-runtime/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index 59ec7f09f939c..40469596d1cb2 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -79,7 +79,7 @@ under the License. io.netty netty-all - 4.0.24.Final + 4.0.25.Final