提交 9d7acf36 编写于 作者: U Ufuk Celebi

[FLINK-1350] [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.
上级 1930678f
......@@ -460,12 +460,13 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
public InterestingProperties getInterestingProperties() {
return this.intProps;
}
@Override
public long getEstimatedOutputSize() {
return this.estimatedOutputSize;
}
@Override
public long getEstimatedNumRecords() {
return this.estimatedNumRecords;
}
......@@ -478,6 +479,7 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
this.estimatedNumRecords = estimatedNumRecords;
}
@Override
public float getEstimatedAvgWidthPerOutputRecord() {
if (this.estimatedOutputSize > 0 && this.estimatedNumRecords > 0) {
return ((float) this.estimatedOutputSize) / this.estimatedNumRecords;
......@@ -941,6 +943,7 @@ public abstract class OptimizerNode implements Visitable<OptimizerNode>, Estimat
if (this.closedBranchingNodes == null) {
this.closedBranchingNodes = new HashSet<OptimizerNode>();
}
this.closedBranchingNodes.add(alreadyClosed);
}
......
......@@ -105,6 +105,7 @@ public class Channel implements EstimateProvider, Cloneable, DumpableConnection<
*
* @return The source.
*/
@Override
public PlanNode getSource() {
return this.source;
}
......
......@@ -381,8 +381,7 @@ public abstract class PlanNode implements Visitable<PlanNode>, DumpableNode<Plan
public List<Channel> getOutgoingChannels() {
return this.outChannels;
}
// --------------------------------------------------------------------------------------------
// Miscellaneous
// --------------------------------------------------------------------------------------------
......
......@@ -1068,10 +1068,8 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
default:
throw new RuntimeException("Unknown runtime ship strategy: " + channel.getShipStrategy());
}
targetVertex.connectNewDataSetAsInput(sourceVertex, distributionPattern);
// sourceVertex.conn/ectTo(targetVertex, channelType, distributionPattern);
// -------------- configure the source task's ship strategy strategies in task config --------------
final int outputIndex = sourceConfig.getNumOutputs();
......@@ -1140,6 +1138,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
final TempMode tm = channel.getTempMode();
boolean needsMemory = false;
// Don't add a pipeline breaker if the data exchange is already blocking.
if (tm.breaksPipeline()) {
config.setInputAsynchronouslyMaterialized(inputNum, true);
needsMemory = true;
......
......@@ -18,21 +18,29 @@
package org.apache.flink.compiler;
import static org.junit.Assert.*;
import org.apache.flink.compiler.testfunctions.IdentityMapper;
import org.apache.flink.compiler.testfunctions.SelectOneReducer;
import org.junit.Test;
import org.apache.flink.api.common.Plan;
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.io.DiscardingOutputFormat;
import org.apache.flink.api.java.operators.IterativeDataSet;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.compiler.plan.BulkIterationPlanNode;
import org.apache.flink.compiler.plan.DualInputPlanNode;
import org.apache.flink.compiler.plan.OptimizedPlan;
import org.apache.flink.compiler.plan.PlanNode;
import org.apache.flink.compiler.plan.SingleInputPlanNode;
import org.apache.flink.compiler.plan.SinkPlanNode;
import org.apache.flink.compiler.plan.SourcePlanNode;
import org.apache.flink.compiler.testfunctions.IdentityMapper;
import org.apache.flink.compiler.testfunctions.SelectOneReducer;
import org.apache.flink.configuration.Configuration;
import org.junit.Test;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@SuppressWarnings("serial")
public class PipelineBreakerTest extends CompilerTestBase {
......@@ -42,21 +50,21 @@ public class PipelineBreakerTest extends CompilerTestBase {
try {
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
env.setDegreeOfParallelism(64);
DataSet<Long> source = env.generateSequence(1, 10).map(new IdentityMapper<Long>());
DataSet<Long> result = source.map(new IdentityMapper<Long>())
.map(new IdentityMapper<Long>())
.withBroadcastSet(source, "bc");
.map(new IdentityMapper<Long>())
.withBroadcastSet(source, "bc");
result.print();
Plan p = env.createProgramPlan();
OptimizedPlan op = compileNoStats(p);
SinkPlanNode sink = op.getDataSinks().iterator().next();
SingleInputPlanNode mapper = (SingleInputPlanNode) sink.getInput().getSource();
assertTrue(mapper.getInput().getTempMode().breaksPipeline());
}
catch (Exception e) {
......@@ -64,33 +72,33 @@ public class PipelineBreakerTest extends CompilerTestBase {
fail(e.getMessage());
}
}
@Test
public void testPipelineBreakerBroadcastedAllReduce() {
try {
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
env.setDegreeOfParallelism(64);
DataSet<Long> sourceWithMapper = env.generateSequence(1, 10).map(new IdentityMapper<Long>());
DataSet<Long> bcInput1 = sourceWithMapper
.map(new IdentityMapper<Long>())
.reduce(new SelectOneReducer<Long>());
.map(new IdentityMapper<Long>())
.reduce(new SelectOneReducer<Long>());
DataSet<Long> bcInput2 = env.generateSequence(1, 10);
DataSet<Long> result = sourceWithMapper
.map(new IdentityMapper<Long>())
.withBroadcastSet(bcInput1, "bc1")
.withBroadcastSet(bcInput2, "bc2");
.withBroadcastSet(bcInput1, "bc1")
.withBroadcastSet(bcInput2, "bc2");
result.print();
Plan p = env.createProgramPlan();
OptimizedPlan op = compileNoStats(p);
SinkPlanNode sink = op.getDataSinks().iterator().next();
SingleInputPlanNode mapper = (SingleInputPlanNode) sink.getInput().getSource();
assertTrue(mapper.getInput().getTempMode().breaksPipeline());
}
catch (Exception e) {
......@@ -98,39 +106,39 @@ public class PipelineBreakerTest extends CompilerTestBase {
fail(e.getMessage());
}
}
@Test
public void testPipelineBreakerBroadcastedPartialSolution() {
try {
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
env.setDegreeOfParallelism(64);
DataSet<Long> initialSource = env.generateSequence(1, 10);
IterativeDataSet<Long> iteration = initialSource.iterate(100);
DataSet<Long> sourceWithMapper = env.generateSequence(1, 10).map(new IdentityMapper<Long>());
DataSet<Long> bcInput1 = sourceWithMapper
.map(new IdentityMapper<Long>())
.reduce(new SelectOneReducer<Long>());
.map(new IdentityMapper<Long>())
.reduce(new SelectOneReducer<Long>());
DataSet<Long> result = sourceWithMapper
.map(new IdentityMapper<Long>())
.withBroadcastSet(iteration, "bc2")
.withBroadcastSet(bcInput1, "bc1");
.withBroadcastSet(iteration, "bc2")
.withBroadcastSet(bcInput1, "bc1");
iteration.closeWith(result).print();
Plan p = env.createProgramPlan();
OptimizedPlan op = compileNoStats(p);
SinkPlanNode sink = op.getDataSinks().iterator().next();
BulkIterationPlanNode iterationPlanNode = (BulkIterationPlanNode) sink.getInput().getSource();
SingleInputPlanNode mapper = (SingleInputPlanNode) iterationPlanNode.getRootOfStepFunction();
assertTrue(mapper.getInput().getTempMode().breaksPipeline());
}
catch (Exception e) {
......@@ -138,98 +146,98 @@ public class PipelineBreakerTest extends CompilerTestBase {
fail(e.getMessage());
}
}
@Test
public void testPilelineBreakerWithCross() {
try {
{
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
env.setDegreeOfParallelism(64);
DataSet<Long> initialSource = env.generateSequence(1, 10);
Configuration conf= new Configuration();
Configuration conf = new Configuration();
conf.setString(PactCompiler.HINT_LOCAL_STRATEGY, PactCompiler.HINT_LOCAL_STRATEGY_NESTEDLOOP_BLOCKED_OUTER_FIRST);
initialSource
.map(new IdentityMapper<Long>())
.cross(initialSource).withParameters(conf)
.print();
.map(new IdentityMapper<Long>())
.cross(initialSource).withParameters(conf)
.print();
Plan p = env.createProgramPlan();
OptimizedPlan op = compileNoStats(p);
SinkPlanNode sink = op.getDataSinks().iterator().next();
DualInputPlanNode mapper = (DualInputPlanNode) sink.getInput().getSource();
assertTrue(mapper.getInput1().getTempMode().breaksPipeline());
}
{
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
env.setDegreeOfParallelism(64);
DataSet<Long> initialSource = env.generateSequence(1, 10);
Configuration conf= new Configuration();
Configuration conf = new Configuration();
conf.setString(PactCompiler.HINT_LOCAL_STRATEGY, PactCompiler.HINT_LOCAL_STRATEGY_NESTEDLOOP_BLOCKED_OUTER_SECOND);
initialSource
.map(new IdentityMapper<Long>())
.cross(initialSource).withParameters(conf)
.print();
.map(new IdentityMapper<Long>())
.cross(initialSource).withParameters(conf)
.print();
Plan p = env.createProgramPlan();
OptimizedPlan op = compileNoStats(p);
SinkPlanNode sink = op.getDataSinks().iterator().next();
DualInputPlanNode mapper = (DualInputPlanNode) sink.getInput().getSource();
assertTrue(mapper.getInput2().getTempMode().breaksPipeline());
}
{
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
env.setDegreeOfParallelism(64);
DataSet<Long> initialSource = env.generateSequence(1, 10);
Configuration conf= new Configuration();
Configuration conf = new Configuration();
conf.setString(PactCompiler.HINT_LOCAL_STRATEGY, PactCompiler.HINT_LOCAL_STRATEGY_NESTEDLOOP_STREAMED_OUTER_FIRST);
initialSource
.map(new IdentityMapper<Long>())
.cross(initialSource).withParameters(conf)
.print();
.map(new IdentityMapper<Long>())
.cross(initialSource).withParameters(conf)
.print();
Plan p = env.createProgramPlan();
OptimizedPlan op = compileNoStats(p);
SinkPlanNode sink = op.getDataSinks().iterator().next();
DualInputPlanNode mapper = (DualInputPlanNode) sink.getInput().getSource();
assertTrue(mapper.getInput1().getTempMode().breaksPipeline());
}
{
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
env.setDegreeOfParallelism(64);
DataSet<Long> initialSource = env.generateSequence(1, 10);
Configuration conf= new Configuration();
Configuration conf = new Configuration();
conf.setString(PactCompiler.HINT_LOCAL_STRATEGY, PactCompiler.HINT_LOCAL_STRATEGY_NESTEDLOOP_STREAMED_OUTER_SECOND);
initialSource
.map(new IdentityMapper<Long>())
.cross(initialSource).withParameters(conf)
.print();
.map(new IdentityMapper<Long>())
.cross(initialSource).withParameters(conf)
.print();
Plan p = env.createProgramPlan();
OptimizedPlan op = compileNoStats(p);
SinkPlanNode sink = op.getDataSinks().iterator().next();
DualInputPlanNode mapper = (DualInputPlanNode) sink.getInput().getSource();
assertTrue(mapper.getInput2().getTempMode().breaksPipeline());
}
}
......
......@@ -122,8 +122,7 @@ public final class ConfigConstants {
* The key for the config parameter defining whether the memory manager allocates memory lazy.
*/
public static final String TASK_MANAGER_MEMORY_LAZY_ALLOCATION_KEY = "taskmanager.memory.lazyalloc";
/**
* The config parameter defining the number of buffers used in the network stack. This defines the
* number of possible tasks and shuffles.
......@@ -135,6 +134,12 @@ public final class ConfigConstants {
*/
public static final String TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY = "taskmanager.network.bufferSizeInBytes";
/**
* The implementation to use for spillable/spilled intermediate results, which have both
* synchronous and asynchronous implementations: "sync" or "async".
*/
public static final String TASK_MANAGER_NETWORK_DEFAULT_IO_MODE = "taskmanager.network.defaultIOMode";
/**
* The config parameter defining the number of task slots of a task manager.
*/
......@@ -462,6 +467,12 @@ public final class ConfigConstants {
*/
public static final int DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE = 32768;
/**
* The implementation to use for spillable/spilled intermediate results, which have both
* synchronous and asynchronous implementations: "sync" or "async".
*/
public static final String DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE = "sync";
/**
* Flag indicating whether to start a thread, which repeatedly logs the memory usage of the JVM.
*/
......
......@@ -174,6 +174,12 @@ public class AbstractID implements IOReadableWritable, Comparable<AbstractID>, j
longToByteArray(this.upperPart, ba, SIZE_OF_LONG);
return StringUtils.byteToHexString(ba);
}
public String toShortString() {
final byte[] ba = new byte[SIZE_OF_LONG];
longToByteArray(upperPart, ba, 0);
return StringUtils.byteToHexString(ba);
}
@Override
public int compareTo(AbstractID o) {
......
......@@ -108,5 +108,7 @@ public abstract class Tuple implements java.io.Serializable {
private static final Class<?>[] CLASSES = new Class<?>[] {
Tuple1.class, Tuple2.class, Tuple3.class, Tuple4.class, Tuple5.class, Tuple6.class, Tuple7.class, Tuple8.class, Tuple9.class, Tuple10.class, Tuple11.class, Tuple12.class, Tuple13.class, Tuple14.class, Tuple15.class, Tuple16.class, Tuple17.class, Tuple18.class, Tuple19.class, Tuple20.class, Tuple21.class, Tuple22.class, Tuple23.class, Tuple24.class, Tuple25.class
};
// END_OF_TUPLE_DEPENDENT_CODE
}
......@@ -90,7 +90,7 @@ under the License.
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
<version>4.0.24.Final</version>
<version>4.0.26.Final</version>
</dependency>
<dependency>
......
/*
* 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.deployment;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.executiongraph.Execution;
import org.apache.flink.runtime.executiongraph.ExecutionEdge;
import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
import org.apache.flink.runtime.instance.Instance;
import org.apache.flink.runtime.instance.SimpleSlot;
import org.apache.flink.runtime.io.network.ConnectionID;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.io.network.partition.consumer.InputChannel;
import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.Serializable;
import java.util.Arrays;
import static com.google.common.base.Preconditions.checkNotNull;
/**
* Deployment descriptor for a single input channel instance.
*
* <p> Each input channel consumes a single subpartition. The index of the subpartition to consume
* is part of the {@link InputGateDeploymentDescriptor} as it is the same for each input channel of
* the respective input gate.
*
* @see InputChannel
* @see SingleInputGate
*/
public class InputChannelDeploymentDescriptor implements Serializable {
private static Logger LOG = LoggerFactory.getLogger(InputChannelDeploymentDescriptor.class);
/** The ID of the partition the input channel is going to consume. */
private final ResultPartitionID consumedPartitionId;
/** The location of the partition the input channel is going to consume. */
private final ResultPartitionLocation consumedPartitionLocation;
public InputChannelDeploymentDescriptor(
ResultPartitionID consumedPartitionId,
ResultPartitionLocation consumedPartitionLocation) {
this.consumedPartitionId = checkNotNull(consumedPartitionId);
this.consumedPartitionLocation = checkNotNull(consumedPartitionLocation);
}
public ResultPartitionID getConsumedPartitionId() {
return consumedPartitionId;
}
public ResultPartitionLocation getConsumedPartitionLocation() {
return consumedPartitionLocation;
}
@Override
public String toString() {
return String.format("InputChannelDeploymentDescriptor [consumed partition id: %s, " +
"consumed partition location: %s]",
consumedPartitionId, consumedPartitionLocation);
}
// ------------------------------------------------------------------------
/**
* Creates an input channel deployment descriptor for each partition.
*/
public static InputChannelDeploymentDescriptor[] fromEdges(
ExecutionEdge[] edges, SimpleSlot consumerSlot) {
final InputChannelDeploymentDescriptor[] icdd = new InputChannelDeploymentDescriptor[edges.length];
// Each edge is connected to a different result partition
for (int i = 0; i < edges.length; i++) {
final IntermediateResultPartition consumedPartition = edges[i].getSource();
final Execution producer = consumedPartition.getProducer().getCurrentExecutionAttempt();
final ExecutionState producerState = producer.getState();
final SimpleSlot producerSlot = producer.getAssignedResource();
final ResultPartitionLocation partitionLocation;
// The producing task needs to be RUNNING or already FINISHED
if (consumedPartition.isConsumable() && producerSlot != null &&
(producerState == ExecutionState.RUNNING
|| producerState == ExecutionState.FINISHED)) {
final Instance partitionInstance = producerSlot.getInstance();
if (partitionInstance.equals(consumerSlot.getInstance())) {
// Consuming task is deployed to the same instance as the partition => local
partitionLocation = ResultPartitionLocation.createLocal();
}
else {
// Different instances => remote
final ConnectionID connectionId = new ConnectionID(
partitionInstance.getInstanceConnectionInfo(),
consumedPartition.getIntermediateResult().getConnectionIndex());
partitionLocation = ResultPartitionLocation.createRemote(connectionId);
}
}
else {
// The producing task might not have registered the partition yet
partitionLocation = ResultPartitionLocation.createUnknown();
}
final ResultPartitionID consumedPartitionId = new ResultPartitionID(
consumedPartition.getPartitionId(), producer.getAttemptId());
icdd[i] = new InputChannelDeploymentDescriptor(
consumedPartitionId, partitionLocation);
}
LOG.debug("Created {} from edges {}.", Arrays.toString(icdd), Arrays.toString(edges));
return icdd;
}
}
......@@ -18,85 +18,72 @@
package org.apache.flink.runtime.deployment;
import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
import org.apache.flink.runtime.jobgraph.DistributionPattern;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import java.io.IOException;
import java.io.Serializable;
import java.util.Arrays;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
/**
* A partition consumer deployment descriptor combines information of all partitions, which are
* consumed by a single reader.
* Deployment descriptor for a single input gate instance.
*
* <p> Each input gate consumes partitions of a single intermediate result. The consumed
* subpartition index is the same for each consumed partition.
*
* @see SingleInputGate
*/
public class PartitionConsumerDeploymentDescriptor implements IOReadableWritable, Serializable {
public class InputGateDeploymentDescriptor implements Serializable {
private IntermediateDataSetID resultId;
/**
* The ID of the consumed intermediate result. Each input gate consumes partitions of the
* intermediate result specified by this ID. This ID also identifies the input gate at the
* consuming task.
*/
private final IntermediateDataSetID consumedResultId;
private PartitionInfo[] partitions;
/**
* The index of the consumed subpartition of each consumed partition. This index depends on the
* {@link DistributionPattern} and the subtask indices of the producing and consuming task.
*/
private final int consumedSubpartitionIndex;
private int queueIndex;
/** An input channel for each consumed subpartition. */
private final InputChannelDeploymentDescriptor[] inputChannels;
public PartitionConsumerDeploymentDescriptor() {
}
public PartitionConsumerDeploymentDescriptor(IntermediateDataSetID resultId, PartitionInfo[] partitions, int queueIndex) {
this.resultId = resultId;
this.partitions = partitions;
this.queueIndex = queueIndex;
}
public InputGateDeploymentDescriptor(
IntermediateDataSetID consumedResultId,
int consumedSubpartitionIndex,
InputChannelDeploymentDescriptor[] inputChannels) {
// ------------------------------------------------------------------------
// Properties
// ------------------------------------------------------------------------
this.consumedResultId = checkNotNull(consumedResultId);
public IntermediateDataSetID getResultId() {
return resultId;
}
checkArgument(consumedSubpartitionIndex >= 0);
this.consumedSubpartitionIndex = consumedSubpartitionIndex;
public PartitionInfo[] getPartitions() {
return partitions;
this.inputChannels = checkNotNull(inputChannels);
}
public int getQueueIndex() {
return queueIndex;
public IntermediateDataSetID getConsumedResultId() {
return consumedResultId;
}
// ------------------------------------------------------------------------
// Serialization
// ------------------------------------------------------------------------
@Override
public void write(DataOutputView out) throws IOException {
resultId.write(out);
out.writeInt(partitions.length);
for (PartitionInfo partition : partitions) {
partition.write(out);
}
out.writeInt(queueIndex);
public int getConsumedSubpartitionIndex() {
return consumedSubpartitionIndex;
}
@Override
public void read(DataInputView in) throws IOException {
resultId = new IntermediateDataSetID();
resultId.read(in);
partitions = new PartitionInfo[in.readInt()];
for (int i = 0; i < partitions.length; i++) {
partitions[i] = new PartitionInfo();
partitions[i].read(in);
}
this.queueIndex = in.readInt();
public InputChannelDeploymentDescriptor[] getInputChannelDeploymentDescriptors() {
return inputChannels;
}
@Override
public String toString() {
return String.format("PartitionConsumerDeploymentDescriptor(ResultID: %s, " +
"Queue index: %d, Partitions: %s)", resultId, queueIndex,
Arrays.toString(partitions));
return String.format("InputGateDeploymentDescriptor [result id: %s, " +
"consumed subpartition index: %d, input channels: %s]",
consumedResultId.toShortString(), consumedSubpartitionIndex,
Arrays.toString(inputChannels));
}
}
/*
* 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.deployment;
import org.apache.flink.runtime.executiongraph.Execution;
import org.apache.flink.runtime.executiongraph.IntermediateResult;
import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
import org.apache.flink.runtime.instance.InstanceConnectionInfo;
import org.apache.flink.runtime.io.network.ConnectionID;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import static com.google.common.base.Preconditions.checkNotNull;
/**
* Partial deployment descriptor for a single input channel instance.
*
* <p> This deployment descriptor is created in {@link Execution#scheduleOrUpdateConsumers(java.util.List)},
* if the consumer instance is not yet clear. Once the instance on which the consumer runs is known,
* the deployment descriptor is updated by completing the partition location.
*/
public class PartialInputChannelDeploymentDescriptor {
/** The result ID identifies the input gate to update. */
private final IntermediateDataSetID resultId;
/** The partition ID identifies the input channel to update. */
private final ResultPartitionID partitionID;
/** The partition connection info. */
private final InstanceConnectionInfo partitionConnectionInfo;
/** The partition connection index. */
private final int partitionConnectionIndex;
public PartialInputChannelDeploymentDescriptor(
IntermediateDataSetID resultId,
ResultPartitionID partitionID,
InstanceConnectionInfo partitionConnectionInfo,
int partitionConnectionIndex) {
this.resultId = checkNotNull(resultId);
this.partitionID = checkNotNull(partitionID);
this.partitionConnectionInfo = checkNotNull(partitionConnectionInfo);
this.partitionConnectionIndex = partitionConnectionIndex;
}
/**
* Creates a channel deployment descriptor by completing the partition location.
*
* @see InputChannelDeploymentDescriptor
*/
public InputChannelDeploymentDescriptor createInputChannelDeploymentDescriptor(
Execution consumerExecution) {
checkNotNull(consumerExecution, "Consumer execution null");
InstanceConnectionInfo consumerConnectionInfo = consumerExecution.getAssignedResourceLocation();
checkNotNull(consumerConnectionInfo, "Consumer connection info null");
final ResultPartitionLocation partitionLocation;
if (consumerConnectionInfo.equals(partitionConnectionInfo)) {
partitionLocation = ResultPartitionLocation.createLocal();
}
else {
partitionLocation = ResultPartitionLocation.createRemote(
new ConnectionID(partitionConnectionInfo, partitionConnectionIndex));
}
return new InputChannelDeploymentDescriptor(partitionID, partitionLocation);
}
public IntermediateDataSetID getResultId() {
return resultId;
}
// ------------------------------------------------------------------------
/**
* Creates a partial input channel for the given partition and producing task.
*/
public static PartialInputChannelDeploymentDescriptor fromEdge(
IntermediateResultPartition partition,
Execution producer) {
final ResultPartitionID partitionId = new ResultPartitionID(
partition.getPartitionId(), producer.getAttemptId());
final IntermediateResult result = partition.getIntermediateResult();
final IntermediateDataSetID resultId = result.getId();
final InstanceConnectionInfo partitionConnectionInfo = producer.getAssignedResourceLocation();
final int partitionConnectionIndex = result.getConnectionIndex();
return new PartialInputChannelDeploymentDescriptor(
resultId, partitionId, partitionConnectionInfo, partitionConnectionIndex);
}
}
/*
* 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.deployment;
import org.apache.flink.runtime.executiongraph.Execution;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.executiongraph.ExecutionEdge;
import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
import org.apache.flink.runtime.instance.InstanceConnectionInfo;
import org.apache.flink.runtime.io.network.RemoteAddress;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
/**
* This class contains the partial partition info which is created if the consumer instance is not
* yet clear. Once the instance on which the consumer runs is known, the complete partition info
* can be computed.
*/
public class PartialPartitionInfo {
private final IntermediateDataSetID intermediateDataSetID;
private final IntermediateResultPartitionID partitionID;
private final ExecutionAttemptID producerExecutionID;
private final InstanceConnectionInfo producerInstanceConnectionInfo;
private final int partitionConnectionIndex;
public PartialPartitionInfo(IntermediateDataSetID intermediateDataSetID,
IntermediateResultPartitionID partitionID,
ExecutionAttemptID executionID,
InstanceConnectionInfo producerInstanceConnectionInfo,
int partitionConnectionIndex) {
this.intermediateDataSetID = intermediateDataSetID;
this.partitionID = partitionID;
this.producerExecutionID = executionID;
this.producerInstanceConnectionInfo = producerInstanceConnectionInfo;
this.partitionConnectionIndex = partitionConnectionIndex;
}
public PartitionInfo createPartitionInfo(Execution consumerExecution) throws IllegalStateException {
if(consumerExecution != null){
PartitionInfo.PartitionLocation producerLocation;
RemoteAddress resolvedProducerAddress;
if(consumerExecution.getAssignedResourceLocation().equals(
producerInstanceConnectionInfo)) {
resolvedProducerAddress = null;
producerLocation = PartitionInfo.PartitionLocation.LOCAL;
} else {
resolvedProducerAddress = new RemoteAddress(producerInstanceConnectionInfo,
partitionConnectionIndex);
producerLocation = PartitionInfo.PartitionLocation.REMOTE;
}
return new PartitionInfo(partitionID, producerExecutionID, producerLocation,
resolvedProducerAddress);
} else {
throw new RuntimeException("Cannot create partition info, because consumer execution " +
"is null.");
}
}
public IntermediateDataSetID getIntermediateDataSetID() {
return intermediateDataSetID;
}
public static PartialPartitionInfo fromEdge(final ExecutionEdge edge){
IntermediateResultPartition partition = edge.getSource();
IntermediateResultPartitionID partitionID = edge.getSource().getPartitionId();
IntermediateDataSetID intermediateDataSetID = partition.getIntermediateResult().getId();
Execution producer = partition.getProducer().getCurrentExecutionAttempt();
ExecutionAttemptID producerExecutionID = producer.getAttemptId();
return new PartialPartitionInfo(intermediateDataSetID, partitionID, producerExecutionID,
producer.getAssignedResourceLocation(),
partition.getIntermediateResult().getConnectionIndex());
}
}
/*
* 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.deployment;
import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.executiongraph.Execution;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.executiongraph.ExecutionEdge;
import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
import org.apache.flink.runtime.instance.SimpleSlot;
import org.apache.flink.runtime.io.network.RemoteAddress;
import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.io.Serializable;
import static com.google.common.base.Preconditions.checkNotNull;
/**
* A partition info instance contains all information necessary for a reader to create an input
* channel to request a partition at runtime.
*/
public class PartitionInfo implements IOReadableWritable, Serializable {
private static Logger LOG = LoggerFactory.getLogger(PartitionInfo.class);
public enum PartitionLocation {
LOCAL, REMOTE, UNKNOWN
}
private final IntermediateResultPartitionID partitionId;
private ExecutionAttemptID producerExecutionId;
private PartitionLocation producerLocation;
private RemoteAddress producerAddress; // != null, iff known remote producer
public PartitionInfo(IntermediateResultPartitionID partitionId, ExecutionAttemptID producerExecutionId, PartitionLocation producerLocation, RemoteAddress producerAddress) {
this.partitionId = checkNotNull(partitionId);
this.producerExecutionId = checkNotNull(producerExecutionId);
this.producerLocation = checkNotNull(producerLocation);
this.producerAddress = producerAddress;
}
public PartitionInfo() {
this.partitionId = new IntermediateResultPartitionID();
this.producerExecutionId = new ExecutionAttemptID();
this.producerLocation = PartitionLocation.UNKNOWN;
this.producerAddress = null;
}
// ------------------------------------------------------------------------
// Properties
// ------------------------------------------------------------------------
public IntermediateResultPartitionID getPartitionId() {
return partitionId;
}
public ExecutionAttemptID getProducerExecutionId() {
return producerExecutionId;
}
public PartitionLocation getProducerLocation() {
return producerLocation;
}
public RemoteAddress getProducerAddress() {
return producerAddress;
}
// ------------------------------------------------------------------------
// Serialization
// ------------------------------------------------------------------------
@Override
public void write(DataOutputView out) throws IOException {
partitionId.write(out);
producerExecutionId.write(out);
out.writeInt(producerLocation.ordinal());
if (producerLocation == PartitionLocation.REMOTE) {
producerAddress.write(out);
}
}
@Override
public void read(DataInputView in) throws IOException {
partitionId.read(in);
producerExecutionId.read(in);
producerLocation = PartitionLocation.values()[in.readInt()];
if (producerLocation == PartitionLocation.REMOTE) {
producerAddress = new RemoteAddress();
producerAddress.read(in);
}
}
// ------------------------------------------------------------------------
public static PartitionInfo fromEdge(ExecutionEdge edge, SimpleSlot consumerSlot) {
IntermediateResultPartition partition = edge.getSource();
IntermediateResultPartitionID partitionId = partition.getPartitionId();
// Intermediate result partition producer
Execution producer = partition.getProducer().getCurrentExecutionAttempt();
ExecutionAttemptID producerExecutionId = producer.getAttemptId();
RemoteAddress producerAddress = null;
PartitionLocation producerLocation = PartitionLocation.UNKNOWN;
SimpleSlot producerSlot = producer.getAssignedResource();
ExecutionState producerState = producer.getState();
// 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 ||
producerState == ExecutionState.FINISHED)) {
if (producerSlot.getInstance().equals(consumerSlot.getInstance())) {
producerLocation = PartitionLocation.LOCAL;
}
else {
producerAddress = new RemoteAddress(producerSlot.getInstance().getInstanceConnectionInfo(),
partition.getIntermediateResult().getConnectionIndex());
producerLocation = PartitionLocation.REMOTE;
}
}
PartitionInfo partitionInfo = new PartitionInfo(partitionId, producerExecutionId,
producerLocation, producerAddress);
LOG.debug("Create partition info {}.", partitionInfo);
return partitionInfo;
}
public static PartitionInfo[] fromEdges(ExecutionEdge[] edges, SimpleSlot consumerSlot) {
// Every edge consumes a different result partition, which might be of
// local, remote, or unknown location.
PartitionInfo[] partitions = new PartitionInfo[edges.length];
for (int i = 0; i < edges.length; i++) {
partitions[i] = fromEdge(edges[i], consumerSlot);
}
return partitions;
}
@Override
public String toString() {
return String.format("PartitionInfo(PartitionID: %s, ProducerID: %s, " +
"ProducerLocation: %s, ProducerAddress: %s)", partitionId, producerExecutionId,
producerLocation, producerAddress);
}
}
......@@ -18,47 +18,49 @@
package org.apache.flink.runtime.deployment;
import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.runtime.executiongraph.IntermediateResultPartition;
import org.apache.flink.runtime.io.network.partition.ResultPartition;
import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionType;
import java.io.IOException;
import java.io.Serializable;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
/**
* A partition deployment descriptor combines information for a produced intermediate result
* partition.
* Deployment descriptor for a result partition.
*
* @see ResultPartition
*/
public class PartitionDeploymentDescriptor implements IOReadableWritable, Serializable {
public class ResultPartitionDeploymentDescriptor implements Serializable {
/** The ID of the result this partition belongs to. */
private final IntermediateDataSetID resultId;
/** The ID of the partition. */
private final IntermediateResultPartitionID partitionId;
private IntermediateResultPartitionType partitionType;
/** The type of the partition. */
private final ResultPartitionType partitionType;
private int numberOfQueues;
/** The number of subpartitions. */
private final int numberOfSubpartitions;
public PartitionDeploymentDescriptor() {
this.resultId = new IntermediateDataSetID();
this.partitionId = new IntermediateResultPartitionID();
this.numberOfQueues = -1;
}
public ResultPartitionDeploymentDescriptor(
IntermediateDataSetID resultId,
IntermediateResultPartitionID partitionId,
ResultPartitionType partitionType,
int numberOfSubpartitions) {
public PartitionDeploymentDescriptor(IntermediateDataSetID resultId, IntermediateResultPartitionID partitionId, IntermediateResultPartitionType partitionType, int numberOfQueues) {
this.resultId = resultId;
this.partitionId = partitionId;
this.partitionType = partitionType;
this.numberOfQueues = numberOfQueues;
}
this.resultId = checkNotNull(resultId);
this.partitionId = checkNotNull(partitionId);
this.partitionType = checkNotNull(partitionType);
// ------------------------------------------------------------------------
// Properties
// ------------------------------------------------------------------------
checkArgument(numberOfSubpartitions >= 1);
this.numberOfSubpartitions = numberOfSubpartitions;
}
public IntermediateDataSetID getResultId() {
return resultId;
......@@ -68,55 +70,45 @@ public class PartitionDeploymentDescriptor implements IOReadableWritable, Serial
return partitionId;
}
public IntermediateResultPartitionType getPartitionType() {
public ResultPartitionType getPartitionType() {
return partitionType;
}
public int getNumberOfQueues() {
return numberOfQueues;
}
// ------------------------------------------------------------------------
// Serialization
// ------------------------------------------------------------------------
@Override
public void write(DataOutputView out) throws IOException {
resultId.write(out);
partitionId.write(out);
out.writeInt(partitionType.ordinal());
out.writeInt(numberOfQueues);
public int getNumberOfSubpartitions() {
return numberOfSubpartitions;
}
@Override
public void read(DataInputView in) throws IOException {
resultId.read(in);
partitionId.read(in);
partitionType = IntermediateResultPartitionType.values()[in.readInt()];
numberOfQueues = in.readInt();
public String toString() {
return String.format("ResultPartitionDeploymentDescriptor [result id: %s," +
"partition id: %s,partition type: %s]",
resultId, partitionId, partitionType);
}
// ------------------------------------------------------------------------
public static PartitionDeploymentDescriptor fromIntermediateResultPartition(IntermediateResultPartition partition) {
public static ResultPartitionDeploymentDescriptor from(IntermediateResultPartition partition) {
IntermediateResultPartitionID partitionId = partition.getPartitionId();
final IntermediateDataSetID resultId = partition.getIntermediateResult().getId();
final IntermediateResultPartitionID partitionId = partition.getPartitionId();
final ResultPartitionType partitionType = partition.getIntermediateResult().getResultType();
// 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,
// otherwise we have a queue for each consumer sub task.
int numberOfQueues = 1;
// The produced data is partitioned among a number of subpartitions.
//
// If no consumers are known at this point, we use a single subpartition, otherwise we have
// one for each consuming sub task.
int numberOfSubpartitions = 1;
if (!partition.getConsumers().isEmpty() && !partition.getConsumers().get(0).isEmpty()) {
numberOfQueues = partition.getConsumers().get(0).size();
}
return new PartitionDeploymentDescriptor(partition.getIntermediateResult().getId(), partitionId, partition.getIntermediateResult().getResultType(), numberOfQueues);
}
if (partition.getConsumers().size() > 1) {
new IllegalStateException("Currently, only a single consumer group per partition is supported.");
}
@Override
public String toString() {
return String.format("PartitionDeploymentDescriptor(ResultID: %s, partitionID: %s, " +
"Partition type: %s)", resultId, partitionId, partitionType);
numberOfSubpartitions = partition.getConsumers().get(0).size();
}
return new ResultPartitionDeploymentDescriptor(
resultId, partitionId, partitionType, numberOfSubpartitions);
}
}
/*
* 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.deployment;
import org.apache.flink.runtime.io.network.ConnectionID;
import java.io.Serializable;
import static com.google.common.base.Preconditions.checkNotNull;
/**
* Location of a result partition from the perspective of the consuming task.
*
* <p> The location indicates both the instance, on which the partition is produced and the state of
* the producing task. There are three possibilities:
*
* <ol>
* <li><strong>Local:</strong> The partition is available at the same instance on which the
* consuming task is (being) deployed and the producing task has registered the result partition.
*
* <li><strong>Remote:</strong> The result partition is available at a different instance from the
* one, on which the consuming task is (being) deployed and the producing task has registered the
* result partition.
*
* <li><strong>Unknown:</strong> The producing task has not yet registered the result partition.
* When deploying the consuming task, the instance might be known or unknown. In any case, the
* consuming task cannot request it yet. Instead, it will be updated at runtime after the
* producing task is guaranteed to have registered the partition. A producing task is guaranteed
* to have registered the partition after its state has switched to running.
* </ol>
*/
public class ResultPartitionLocation implements Serializable {
/** The type of location for the result partition. */
private final LocationType locationType;
/** The connection ID of a remote result partition. */
private final ConnectionID connectionId;
private enum LocationType {
LOCAL,
REMOTE,
UNKNOWN
}
private ResultPartitionLocation(LocationType locationType, ConnectionID connectionId) {
this.locationType = checkNotNull(locationType);
this.connectionId = connectionId;
}
public static ResultPartitionLocation createRemote(ConnectionID connectionId) {
return new ResultPartitionLocation(LocationType.REMOTE, checkNotNull(connectionId));
}
public static ResultPartitionLocation createLocal() {
return new ResultPartitionLocation(LocationType.LOCAL, null);
}
public static ResultPartitionLocation createUnknown() {
return new ResultPartitionLocation(LocationType.UNKNOWN, null);
}
// ------------------------------------------------------------------------
public boolean isLocal() {
return locationType == LocationType.LOCAL;
}
public boolean isRemote() {
return locationType == LocationType.REMOTE;
}
public boolean isUnknown() {
return locationType == LocationType.UNKNOWN;
}
public ConnectionID getConnectionId() {
return connectionId;
}
@Override
public String toString() {
return "ResultPartitionLocation [" + locationType + (isRemote() ? " [" + connectionId + "]]" : "]");
}
}
......@@ -26,7 +26,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.state.StateHandle;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import static com.google.common.base.Preconditions.checkArgument;
......@@ -49,47 +49,47 @@ public final class TaskDeploymentDescriptor implements Serializable {
private final ExecutionAttemptID executionId;
/** The task's name. */
private String taskName;
private final String taskName;
/** The task's index in the subtask group. */
private int indexInSubtaskGroup;
private final int indexInSubtaskGroup;
/** The number of sub tasks. */
private int numberOfSubtasks;
private final int numberOfSubtasks;
/** The configuration of the job the task belongs to. */
private Configuration jobConfiguration;
private final Configuration jobConfiguration;
/** The task's configuration object. */
private Configuration taskConfiguration;
private final Configuration taskConfiguration;
/** The name of the class containing the task code to be executed. */
private String invokableClassName;
private final String invokableClassName;
/** The list of produced intermediate result partition deployment descriptors. */
private List<PartitionDeploymentDescriptor> producedPartitions;
private final List<ResultPartitionDeploymentDescriptor> producedPartitions;
/** The list of consumed intermediate result partitions. */
private List<PartitionConsumerDeploymentDescriptor> consumedPartitions;
private final List<InputGateDeploymentDescriptor> inputGates;
private int targetSlotNumber;
private final int targetSlotNumber;
/** The list of JAR files required to run this task. */
private final List<BlobKey> requiredJarFiles;
private StateHandle operatorStates;
/**
* Constructs a task deployment descriptor.
*/
public TaskDeploymentDescriptor(
JobID jobID, JobVertexID vertexID, ExecutionAttemptID executionId, String taskName,
int indexInSubtaskGroup, int numberOfSubtasks, Configuration jobConfiguration,
JobID jobID, JobVertexID vertexID, ExecutionAttemptID executionId, String taskName,
int indexInSubtaskGroup, int numberOfSubtasks, Configuration jobConfiguration,
Configuration taskConfiguration, String invokableClassName,
List<PartitionDeploymentDescriptor> producedPartitions,
List<PartitionConsumerDeploymentDescriptor> consumedPartitions,
List<BlobKey> requiredJarFiles, int targetSlotNumber){
List<ResultPartitionDeploymentDescriptor> producedPartitions,
List<InputGateDeploymentDescriptor> inputGates,
List<BlobKey> requiredJarFiles, int targetSlotNumber) {
this.jobID = checkNotNull(jobID);
this.vertexID = checkNotNull(vertexID);
......@@ -103,37 +103,25 @@ public final class TaskDeploymentDescriptor implements Serializable {
this.taskConfiguration = checkNotNull(taskConfiguration);
this.invokableClassName = checkNotNull(invokableClassName);
this.producedPartitions = checkNotNull(producedPartitions);
this.consumedPartitions = checkNotNull(consumedPartitions);
this.inputGates = checkNotNull(inputGates);
this.requiredJarFiles = checkNotNull(requiredJarFiles);
checkArgument(targetSlotNumber >= 0);
this.targetSlotNumber = targetSlotNumber;
}
/**
* Default constructor for serialization/deserialization.
*/
public TaskDeploymentDescriptor() {
this.jobID = new JobID();
this.vertexID = new JobVertexID();
this.executionId = new ExecutionAttemptID();
this.jobConfiguration = new Configuration();
this.taskConfiguration = new Configuration();
this.producedPartitions = new ArrayList<PartitionDeploymentDescriptor>();
this.consumedPartitions = new ArrayList<PartitionConsumerDeploymentDescriptor>();
this.requiredJarFiles = new ArrayList<BlobKey>();
}
public TaskDeploymentDescriptor(
JobID jobID, JobVertexID vertexID, ExecutionAttemptID executionId, String taskName,
int indexInSubtaskGroup, int numberOfSubtasks, Configuration jobConfiguration,
Configuration taskConfiguration, String invokableClassName,
List<PartitionDeploymentDescriptor> producedPartitions,
List<PartitionConsumerDeploymentDescriptor> consumedPartitions,
List<BlobKey> requiredJarFiles, int targetSlotNumber, StateHandle operatorStates) {
List<ResultPartitionDeploymentDescriptor> producedPartitions,
List<InputGateDeploymentDescriptor> inputGates,
List<BlobKey> requiredJarFiles, int targetSlotNumber,
StateHandle operatorStates) {
this(jobID, vertexID, executionId, taskName, indexInSubtaskGroup, numberOfSubtasks,
jobConfiguration, taskConfiguration, invokableClassName, producedPartitions,
consumedPartitions, requiredJarFiles, targetSlotNumber);
inputGates, requiredJarFiles, targetSlotNumber);
setOperatorState(operatorStates);
}
......@@ -164,7 +152,7 @@ public final class TaskDeploymentDescriptor implements Serializable {
/**
* Returns the task's index in the subtask group.
*
*
* @return the task's index in the subtask group
*/
public int getIndexInSubtaskGroup() {
......@@ -177,10 +165,10 @@ public final class TaskDeploymentDescriptor implements Serializable {
public int getNumberOfSubtasks() {
return numberOfSubtasks;
}
/**
* Gets the number of the slot into which the task is to be deployed.
*
*
* @return The number of the target slot.
*/
public int getTargetSlotNumber() {
......@@ -208,12 +196,12 @@ public final class TaskDeploymentDescriptor implements Serializable {
return invokableClassName;
}
public List<PartitionDeploymentDescriptor> getProducedPartitions() {
public List<ResultPartitionDeploymentDescriptor> getProducedPartitions() {
return producedPartitions;
}
public List<PartitionConsumerDeploymentDescriptor> getConsumedPartitions() {
return consumedPartitions;
public List<InputGateDeploymentDescriptor> getInputGates() {
return inputGates;
}
public List<BlobKey> getRequiredJarFiles() {
......@@ -222,25 +210,26 @@ public final class TaskDeploymentDescriptor implements Serializable {
@Override
public String toString() {
final StringBuilder pddBuilder = new StringBuilder("");
final StringBuilder pcddBuilder = new StringBuilder("");
return String.format("TaskDeploymentDescriptor [job id: %s, job vertex id: %s, " +
"execution id: %s, task name: %s (%d/%d), invokable: %s, " +
"produced partitions: %s, input gates: %s]",
jobID, vertexID, executionId, taskName, indexInSubtaskGroup, numberOfSubtasks,
invokableClassName, collectionToString(producedPartitions),
collectionToString(inputGates));
}
for(PartitionDeploymentDescriptor pdd: producedPartitions) {
pddBuilder.append(pdd);
}
private String collectionToString(Collection<?> collection) {
final StringBuilder strBuilder = new StringBuilder();
strBuilder.append("[");
for(PartitionConsumerDeploymentDescriptor pcdd: consumedPartitions) {
pcddBuilder.append(pcdd);
for (Object elem : collection) {
strBuilder.append(elem.toString());
}
final String strProducedPartitions = pddBuilder.toString();
final String strConsumedPartitions = pcddBuilder.toString();
strBuilder.append("]");
return String.format("TaskDeploymentDescriptor(JobID: %s, JobVertexID: %s, " +
"ExecutionID: %s, Task name: %s, (%d/%d), Invokable: %s, " +
"Produced partitions: %s, Consumed partitions: %s", jobID, vertexID, executionId,
taskName, indexInSubtaskGroup, numberOfSubtasks, invokableClassName,
strProducedPartitions, strConsumedPartitions);
return strBuilder.toString();
}
public void setOperatorState(StateHandle operatorStates) {
......
......@@ -23,7 +23,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.network.api.writer.BufferWriter;
import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobgraph.JobID;
......@@ -130,9 +130,9 @@ public interface Environment {
BroadcastVariableManager getBroadcastVariableManager();
BufferWriter getWriter(int index);
ResultPartitionWriter getWriter(int index);
BufferWriter[] getAllWriters();
ResultPartitionWriter[] getAllWriters();
InputGate getInputGate(int index);
......
......@@ -22,13 +22,14 @@ import akka.actor.ActorRef;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
import org.apache.flink.runtime.deployment.PartitionConsumerDeploymentDescriptor;
import org.apache.flink.runtime.deployment.PartitionDeploymentDescriptor;
import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
import org.apache.flink.runtime.io.network.NetworkEnvironment;
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.api.writer.ResultPartitionWriter;
import org.apache.flink.runtime.io.network.partition.ResultPartition;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
......@@ -92,9 +93,8 @@ public class RuntimeEnvironment implements Environment, Runnable {
private final AtomicBoolean canceled = new AtomicBoolean();
private final IntermediateResultPartition[] producedPartitions;
private final BufferWriter[] writers;
private final ResultPartition[] producedPartitions;
private final ResultPartitionWriter[] writers;
private final SingleInputGate[] inputGates;
......@@ -116,23 +116,27 @@ public class RuntimeEnvironment implements Environment, Runnable {
try {
// Produced intermediate result partitions
final List<PartitionDeploymentDescriptor> partitions = tdd.getProducedPartitions();
final List<ResultPartitionDeploymentDescriptor> partitions = tdd.getProducedPartitions();
this.producedPartitions = new IntermediateResultPartition[partitions.size()];
this.writers = new BufferWriter[partitions.size()];
this.producedPartitions = new ResultPartition[partitions.size()];
this.writers = new ResultPartitionWriter[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));
writers[i] = new BufferWriter(this.producedPartitions[i]);
ResultPartitionDeploymentDescriptor desc = partitions.get(i);
ResultPartitionID partitionId = new ResultPartitionID(desc.getPartitionId(), owner.getExecutionId());
this.producedPartitions[i] = new ResultPartition(owner.getJobID(), partitionId, desc.getPartitionType(), desc.getNumberOfSubpartitions(), networkEnvironment, ioManager);
writers[i] = new ResultPartitionWriter(this.producedPartitions[i]);
}
// Consumed intermediate result partitions
final List<PartitionConsumerDeploymentDescriptor> consumedPartitions = tdd.getConsumedPartitions();
final List<InputGateDeploymentDescriptor> consumedPartitions = tdd.getInputGates();
this.inputGates = new SingleInputGate[consumedPartitions.size()];
for (int i = 0; i < inputGates.length; i++) {
inputGates[i] = SingleInputGate.create(networkEnvironment, consumedPartitions.get(i));
inputGates[i] = SingleInputGate.create(consumedPartitions.get(i), networkEnvironment);
// The input gates are organized by key for task updates/channel updates at runtime
inputGatesById.put(inputGates[i].getConsumedResultId(), inputGates[i]);
......@@ -211,7 +215,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
// Finish the produced partitions
if (producedPartitions != null) {
for (IntermediateResultPartition partition : producedPartitions) {
for (ResultPartition partition : producedPartitions) {
if (partition != null) {
partition.finish();
}
......@@ -340,14 +344,14 @@ public class RuntimeEnvironment implements Environment, Runnable {
}
@Override
public BufferWriter getWriter(int index) {
public ResultPartitionWriter getWriter(int index) {
checkElementIndex(index, writers.length, "Illegal environment writer request.");
return writers[checkElementIndex(index, writers.length)];
}
@Override
public BufferWriter[] getAllWriters() {
public ResultPartitionWriter[] getAllWriters() {
return writers;
}
......@@ -363,7 +367,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
return inputGates;
}
public IntermediateResultPartition[] getProducedPartitions() {
public ResultPartition[] getProducedPartitions() {
return producedPartitions;
}
......
......@@ -20,23 +20,22 @@ package org.apache.flink.runtime.executiongraph;
import akka.actor.ActorRef;
import akka.dispatch.OnComplete;
import static akka.dispatch.Futures.future;
import akka.dispatch.OnFailure;
import akka.pattern.Patterns;
import akka.util.Timeout;
import org.apache.flink.runtime.JobException;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.deployment.PartialPartitionInfo;
import org.apache.flink.runtime.deployment.PartitionInfo;
import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
import org.apache.flink.runtime.deployment.PartialInputChannelDeploymentDescriptor;
import org.apache.flink.runtime.deployment.ResultPartitionLocation;
import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.instance.Instance;
import org.apache.flink.runtime.instance.InstanceConnectionInfo;
import org.apache.flink.runtime.instance.SimpleSlot;
import org.apache.flink.runtime.instance.Instance;
import org.apache.flink.runtime.io.network.RemoteAddress;
import org.apache.flink.runtime.io.network.ConnectionID;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint;
import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
......@@ -44,24 +43,23 @@ import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture;
import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFutureAction;
import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
import org.apache.flink.runtime.messages.TaskManagerMessages;
import org.apache.flink.runtime.messages.TaskManagerMessages.TaskOperationResult;
import org.apache.flink.runtime.state.StateHandle;
import org.apache.flink.util.ExceptionUtils;
import org.slf4j.Logger;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
import static com.google.common.base.Preconditions.checkArgument;
import static akka.dispatch.Futures.future;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.apache.flink.runtime.execution.ExecutionState.CANCELED;
import static org.apache.flink.runtime.execution.ExecutionState.CANCELING;
......@@ -71,6 +69,12 @@ import static org.apache.flink.runtime.execution.ExecutionState.FAILED;
import static org.apache.flink.runtime.execution.ExecutionState.FINISHED;
import static org.apache.flink.runtime.execution.ExecutionState.RUNNING;
import static org.apache.flink.runtime.execution.ExecutionState.SCHEDULED;
import static org.apache.flink.runtime.messages.TaskManagerMessages.CancelTask;
import static org.apache.flink.runtime.messages.TaskManagerMessages.FailIntermediateResultPartitions;
import static org.apache.flink.runtime.messages.TaskManagerMessages.SubmitTask;
import static org.apache.flink.runtime.messages.TaskManagerMessages.UpdateTask;
import static org.apache.flink.runtime.messages.TaskManagerMessages.UpdateTaskSinglePartitionInfo;
import static org.apache.flink.runtime.messages.TaskManagerMessages.createUpdateTaskMultiplePartitionInfos;
/**
* A single execution of a vertex. While an {@link ExecutionVertex} can be executed multiple times (for recovery,
......@@ -113,7 +117,7 @@ public class Execution implements Serializable {
private final FiniteDuration timeout;
private ConcurrentLinkedQueue<PartialPartitionInfo> partialPartitionInfos;
private ConcurrentLinkedQueue<PartialInputChannelDeploymentDescriptor> partialInputChannelDeploymentDescriptors;
private volatile ExecutionState state = CREATED;
......@@ -128,8 +132,6 @@ public class Execution implements Serializable {
// --------------------------------------------------------------------------------------------
public Execution(ExecutionVertex vertex, int attemptNumber, long startTimestamp, FiniteDuration timeout) {
checkArgument(attemptNumber >= 0);
this.vertex = checkNotNull(vertex);
this.attemptId = new ExecutionAttemptID();
......@@ -140,17 +142,17 @@ public class Execution implements Serializable {
this.timeout = timeout;
this.partialPartitionInfos = new ConcurrentLinkedQueue<PartialPartitionInfo>();
this.partialInputChannelDeploymentDescriptors = new ConcurrentLinkedQueue<PartialInputChannelDeploymentDescriptor>();
}
// --------------------------------------------------------------------------------------------
// Properties
// --------------------------------------------------------------------------------------------
public ExecutionVertex getVertex() {
return vertex;
}
public ExecutionAttemptID getAttemptId() {
return attemptId;
}
......@@ -158,15 +160,15 @@ public class Execution implements Serializable {
public int getAttemptNumber() {
return attemptNumber;
}
public ExecutionState getState() {
return state;
}
public SimpleSlot getAssignedResource() {
return assignedResource;
}
public InstanceConnectionInfo getAssignedResourceLocation() {
return assignedResourceLocation;
}
......@@ -196,8 +198,8 @@ public class Execution implements Serializable {
}
assignedResource = null;
partialPartitionInfos.clear();
partialPartitionInfos = null;
partialInputChannelDeploymentDescriptors.clear();
partialInputChannelDeploymentDescriptors = null;
}
// --------------------------------------------------------------------------------------------
......@@ -226,7 +228,7 @@ public class Execution implements Serializable {
// sanity check
if (locationConstraint != null && sharingGroup == null) {
throw new RuntimeException("Trying to schedule with co-location constraint but without slot sharing not allowed.");
throw new RuntimeException("Trying to schedule with co-location constraint but without slot sharing allowed.");
}
if (transitionState(CREATED, SCHEDULED)) {
......@@ -328,7 +330,7 @@ public class Execution implements Serializable {
Instance instance = slot.getInstance();
Future<Object> deployAction = Patterns.ask(instance.getTaskManager(),
new TaskManagerMessages.SubmitTask(deployment), new Timeout(timeout));
new SubmitTask(deployment), new Timeout(timeout));
deployAction.onComplete(new OnComplete<Object>(){
......@@ -432,22 +434,38 @@ public class Execution implements Serializable {
}
}
// TODO This leads to many unnecessary RPC calls in most cases
void scheduleOrUpdateConsumers(List<List<ExecutionEdge>> consumers) {
if (consumers.size() != 1) {
fail(new IllegalStateException("Only one consumer is supported currently."));
void scheduleOrUpdateConsumers(List<List<ExecutionEdge>> allConsumers) {
if (allConsumers.size() != 1) {
fail(new IllegalStateException("Currently, only a single consumer group per partition is supported."));
}
final List<ExecutionEdge> consumer = consumers.get(0);
for (ExecutionEdge edge : consumer) {
for (ExecutionEdge edge : allConsumers.get(0)) {
final ExecutionVertex consumerVertex = edge.getTarget();
final ExecutionState consumerState = consumerVertex.getExecutionState();
final Execution consumer = consumerVertex.getCurrentExecutionAttempt();
final ExecutionState consumerState = consumer.getState();
if (consumerState == CREATED) {
consumerVertex.cachePartitionInfo(PartialPartitionInfo.fromEdge(edge));
final IntermediateResultPartition partition = edge.getSource();
// ----------------------------------------------------------------
// Consumer is created => try to deploy and cache input channel
// descriptors if there is a deployment race
// ----------------------------------------------------------------
if (consumerState == CREATED) {
final Execution partitionExecution = partition.getProducer()
.getCurrentExecutionAttempt();
consumerVertex.cachePartitionInfo(PartialInputChannelDeploymentDescriptor.fromEdge(
partition, partitionExecution));
// When deploying a consuming task, its task deployment descriptor will contain all
// deployment information available at the respective time. It is possible that some
// of the partitions to be consumed have not been created yet. These are updated
// runtime via the update messages.
//
// TODO The current approach may send many update messages even though the consuming
// task has already been deployed with all necessary information. We have to check
// whether this is a problem and fix it, if it is.
future(new Callable<Boolean>(){
@Override
public Boolean call() throws Exception {
......@@ -468,41 +486,64 @@ public class Execution implements Serializable {
consumerVertex.sendPartitionInfos();
}
}
else if (consumerState == RUNNING) {
SimpleSlot consumerSlot = consumerVertex.getCurrentAssignedResource();
ExecutionAttemptID consumerExecutionId = consumerVertex.
getCurrentExecutionAttempt().getAttemptId();
IntermediateResultPartitionID partitionID = edge.getSource().getPartitionId();
int connectionIndex = edge.getSource().getIntermediateResult().getConnectionIndex();
PartitionInfo.PartitionLocation producerLocation;
RemoteAddress producerAddress = null;
if(consumerSlot.getInstance().getInstanceConnectionInfo().equals(
getAssignedResourceLocation())) {
producerLocation = PartitionInfo.PartitionLocation.LOCAL;
} else {
producerLocation = PartitionInfo.PartitionLocation.REMOTE;
producerAddress = new RemoteAddress(getAssignedResourceLocation(),
connectionIndex);
}
// ----------------------------------------------------------------
// Consumer is running => send update message now
// ----------------------------------------------------------------
else {
if (consumerState == RUNNING) {
final SimpleSlot consumerSlot = consumer.getAssignedResource();
PartitionInfo partitionInfo = new PartitionInfo(partitionID, attemptId,
producerLocation, producerAddress);
if (consumerSlot == null) {
// The consumer has been reset concurrently
continue;
}
TaskManagerMessages.UpdateTask updateTaskMessage =
new TaskManagerMessages.UpdateTaskSinglePartitionInfo(consumerExecutionId,
edge.getSource().getIntermediateResult().getId(), partitionInfo);
final Instance consumerInstance = consumerSlot.getInstance();
sendUpdateTaskRpcCall(consumerSlot, updateTaskMessage);
}
else if (consumerState == SCHEDULED || consumerState == DEPLOYING) {
consumerVertex.cachePartitionInfo(PartialPartitionInfo.fromEdge(edge));
final ResultPartitionID partitionId = new ResultPartitionID(
partition.getPartitionId(), attemptId);
// double check to resolve race conditions
if(consumerVertex.getExecutionState() == RUNNING){
consumerVertex.sendPartitionInfos();
final Instance partitionInstance = partition.getProducer()
.getCurrentAssignedResource().getInstance();
final ResultPartitionLocation partitionLocation;
if (consumerInstance.equals(partitionInstance)) {
// Consuming task is deployed to the same instance as the partition => local
partitionLocation = ResultPartitionLocation.createLocal();
}
else {
// Different instances => remote
final ConnectionID connectionId = new ConnectionID(
partitionInstance.getInstanceConnectionInfo(),
partition.getIntermediateResult().getConnectionIndex());
partitionLocation = ResultPartitionLocation.createRemote(connectionId);
}
final InputChannelDeploymentDescriptor descriptor = new InputChannelDeploymentDescriptor(
partitionId, partitionLocation);
final UpdateTask updateTaskMessage = new UpdateTaskSinglePartitionInfo(
consumer.getAttemptId(), partition.getIntermediateResult().getId(), descriptor);
sendUpdateTaskRpcCall(consumerSlot, updateTaskMessage);
}
// ----------------------------------------------------------------
// Consumer is scheduled or deploying => cache input channel
// deployment descriptors and send update message later
// ----------------------------------------------------------------
else if (consumerState == SCHEDULED || consumerState == DEPLOYING) {
final Execution partitionExecution = partition.getProducer()
.getCurrentExecutionAttempt();
consumerVertex.cachePartitionInfo(PartialInputChannelDeploymentDescriptor
.fromEdge(partition, partitionExecution));
// double check to resolve race conditions
if (consumerVertex.getExecutionState() == RUNNING) {
consumerVertex.sendPartitionInfos();
}
}
}
}
......@@ -543,6 +584,23 @@ public class Execution implements Serializable {
if (transitionState(current, FINISHED)) {
try {
if (getVertex().finishAllBlockingPartitions()) {
IntermediateResult[] allResults = getVertex().getJobVertex()
.getProducedDataSets();
LOG.debug("Finished all produced partitions ({}). Scheduling all receivers " +
"of the following datasets: {}.", this, Arrays
.toString(allResults));
// Schedule next batch
for (IntermediateResult result : allResults) {
for (IntermediateResultPartition p : result.getPartitions()) {
scheduleOrUpdateConsumers(p.getConsumers());
}
}
}
assignedResource.releaseSlot();
vertex.getExecutionGraph().deregisterExecution(this);
}
......@@ -612,28 +670,28 @@ public class Execution implements Serializable {
}
}
void cachePartitionInfo(PartialPartitionInfo partitionInfo) {
partialPartitionInfos.add(partitionInfo);
void cachePartitionInfo(PartialInputChannelDeploymentDescriptor partitionInfo) {
partialInputChannelDeploymentDescriptors.add(partitionInfo);
}
void sendPartitionInfos() {
// check if the ExecutionVertex has already been archived and thus cleared the
// partial partition infos queue
if(partialPartitionInfos != null && !partialPartitionInfos.isEmpty()) {
if(partialInputChannelDeploymentDescriptors != null && !partialInputChannelDeploymentDescriptors.isEmpty()) {
PartialPartitionInfo partialPartitionInfo;
PartialInputChannelDeploymentDescriptor partialInputChannelDeploymentDescriptor;
List<IntermediateDataSetID> resultIDs = new ArrayList<IntermediateDataSetID>();
List<PartitionInfo> partitionInfos = new ArrayList<PartitionInfo>();
List<InputChannelDeploymentDescriptor> inputChannelDeploymentDescriptors = new ArrayList<InputChannelDeploymentDescriptor>();
while ((partialPartitionInfo = partialPartitionInfos.poll()) != null) {
resultIDs.add(partialPartitionInfo.getIntermediateDataSetID());
partitionInfos.add(partialPartitionInfo.createPartitionInfo(this));
while ((partialInputChannelDeploymentDescriptor = partialInputChannelDeploymentDescriptors.poll()) != null) {
resultIDs.add(partialInputChannelDeploymentDescriptor.getResultId());
inputChannelDeploymentDescriptors.add(partialInputChannelDeploymentDescriptor.createInputChannelDeploymentDescriptor(this));
}
TaskManagerMessages.UpdateTask updateTaskMessage =
TaskManagerMessages.createUpdateTaskMultiplePartitionInfos(attemptId, resultIDs,
partitionInfos);
UpdateTask updateTaskMessage =
createUpdateTaskMultiplePartitionInfos(attemptId, resultIDs,
inputChannelDeploymentDescriptors);
sendUpdateTaskRpcCall(assignedResource, updateTaskMessage);
}
......@@ -752,7 +810,7 @@ public class Execution implements Serializable {
if (slot != null) {
Future<Object> cancelResult = AkkaUtils.retry(slot.getInstance().getTaskManager(), new
TaskManagerMessages.CancelTask(attemptId), NUM_CANCEL_CALL_TRIES,
CancelTask(attemptId), NUM_CANCEL_CALL_TRIES,
AkkaUtils.globalExecutionContext(), timeout);
cancelResult.onComplete(new OnComplete<Object>() {
......@@ -782,7 +840,7 @@ public class Execution implements Serializable {
if (instance.isAlive()) {
try {
// TODO For some tests this could be a problem when querying too early if all resources were released
instance.getTaskManager().tell(new TaskManagerMessages.FailIntermediateResultPartitions(attemptId), ActorRef.noSender());
instance.getTaskManager().tell(new FailIntermediateResultPartitions(attemptId), ActorRef.noSender());
} catch (Throwable t) {
fail(new Exception("Intermediate result partition could not be failed.", t));
}
......@@ -791,7 +849,8 @@ public class Execution implements Serializable {
}
private void sendUpdateTaskRpcCall(final SimpleSlot consumerSlot,
final TaskManagerMessages.UpdateTask updateTaskMsg) {
final UpdateTask updateTaskMsg) {
if (consumerSlot != null) {
final Instance instance = consumerSlot.getInstance();
......
......@@ -43,4 +43,9 @@ public class ExecutionEdge {
public int getInputNum() {
return inputNum;
}
@Override
public String toString() {
return "ExecutionEdge [" + source + " <=> " + target + "]";
}
}
......@@ -25,6 +25,7 @@ import org.apache.flink.runtime.JobException;
import org.apache.flink.runtime.akka.AkkaUtils;
import org.apache.flink.runtime.blob.BlobKey;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.JobID;
......@@ -175,17 +176,14 @@ public class ExecutionGraph implements Serializable {
* Once this value has reached the number of vertices, the job is done. */
private int nextVertexToFinish;
private ActorContext parentContext;
private ActorRef stateMonitorActor;
private boolean monitoringEnabled;
private long monitoringInterval = 10000;
private long monitoringInterval = 10000;
public ExecutionGraph(JobID jobId, String jobName, Configuration jobConfig, FiniteDuration timeout) {
this(jobId, jobName, jobConfig, timeout, new ArrayList<BlobKey>());
}
......@@ -446,6 +444,7 @@ public class ExecutionGraph implements Serializable {
for (ExecutionJobVertex ejv : getVerticesTopologically()) {
ejv.scheduleAll(scheduler, allowQueuedScheduling);
}
break;
case BACKTRACKING:
......@@ -623,24 +622,24 @@ public class ExecutionGraph implements Serializable {
public void loadOperatorStates(Map<Tuple3<JobVertexID, Integer, Long> , StateHandle> states) {
synchronized (this.progressLock) {
for (Map.Entry<Tuple3<JobVertexID, Integer, Long>, StateHandle> state : states.entrySet()) {
for (Map.Entry<Tuple3<JobVertexID, Integer, Long>, StateHandle> state : states.entrySet())
tasks.get(state.getKey()._1()).getTaskVertices()[state.getKey()._2()].setOperatorState(state.getValue());
}
}
}
public void scheduleOrUpdateConsumers(ExecutionAttemptID executionId, int partitionIndex) {
Execution execution = currentExecutions.get(executionId);
public void scheduleOrUpdateConsumers(ResultPartitionID partitionId) {
final Execution execution = currentExecutions.get(partitionId.getProducerId());
if (execution == null) {
fail(new IllegalStateException("Cannot find execution for execution ID " +
executionId));
partitionId.getPartitionId()));
}
else if(execution.getVertex() == null){
fail(new IllegalStateException("Execution with execution ID " + executionId +
" has no vertex assigned."));
else if (execution.getVertex() == null){
fail(new IllegalStateException("Execution with execution ID " +
partitionId.getPartitionId() + " has no vertex assigned."));
} else {
execution.getVertex().scheduleOrUpdateConsumers(partitionIndex);
execution.getVertex().scheduleOrUpdateConsumers(partitionId);
}
}
......
......@@ -18,13 +18,6 @@
package org.apache.flink.runtime.executiongraph;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.flink.api.common.io.StrictlyLocalAssignment;
import org.apache.flink.core.io.InputSplit;
import org.apache.flink.core.io.InputSplitAssigner;
......@@ -39,13 +32,19 @@ import org.apache.flink.runtime.jobgraph.JobEdge;
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.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class ExecutionJobVertex implements Serializable {
......@@ -82,7 +81,6 @@ public class ExecutionJobVertex implements Serializable {
private InputSplitAssigner splitAssigner;
public ExecutionJobVertex(ExecutionGraph graph, AbstractJobVertex jobVertex,
int defaultParallelism, FiniteDuration timeout) throws JobException {
this(graph, jobVertex, defaultParallelism, timeout, System.currentTimeMillis());
......@@ -118,11 +116,14 @@ public class ExecutionJobVertex implements Serializable {
// create the intermediate results
this.producedDataSets = new IntermediateResult[jobVertex.getNumberOfProducedIntermediateDataSets()];
for (int i = 0; i < jobVertex.getProducedDataSets().size(); i++) {
IntermediateDataSet set = jobVertex.getProducedDataSets().get(i);
this.producedDataSets[i] = new IntermediateResult(set.getId(), this, numTaskVertices);
final IntermediateDataSet result = jobVertex.getProducedDataSets().get(i);
this.producedDataSets[i] = new IntermediateResult(
result.getId(), this, numTaskVertices, result.getResultType());
}
// create all task vertices
for (int i = 0; i < numTaskVertices; i++) {
ExecutionVertex vertex = new ExecutionVertex(this, i, this.producedDataSets, timeout, createTimestamp);
......@@ -374,6 +375,11 @@ public class ExecutionJobVertex implements Serializable {
catch (Throwable t) {
throw new RuntimeException("Re-creating the input split assigner failed: " + t.getMessage(), t);
}
// Reset intermediate results
for (IntermediateResult result : producedDataSets) {
result.resetForNewExecution();
}
}
}
......
......@@ -20,17 +20,19 @@ package org.apache.flink.runtime.executiongraph;
import org.apache.flink.runtime.JobException;
import org.apache.flink.runtime.blob.BlobKey;
import org.apache.flink.runtime.deployment.PartialPartitionInfo;
import org.apache.flink.runtime.deployment.PartitionConsumerDeploymentDescriptor;
import org.apache.flink.runtime.deployment.PartitionDeploymentDescriptor;
import org.apache.flink.runtime.deployment.PartitionInfo;
import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
import org.apache.flink.runtime.deployment.PartialInputChannelDeploymentDescriptor;
import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
import org.apache.flink.runtime.execution.ExecutionState;
import org.apache.flink.runtime.instance.Instance;
import org.apache.flink.runtime.instance.InstanceConnectionInfo;
import org.apache.flink.runtime.instance.SimpleSlot;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.jobgraph.DistributionPattern;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
import org.apache.flink.runtime.jobgraph.JobEdge;
import org.apache.flink.runtime.jobgraph.JobID;
import org.apache.flink.runtime.jobgraph.JobVertexID;
......@@ -46,11 +48,12 @@ import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
import static com.google.common.base.Preconditions.checkElementIndex;
import static org.apache.flink.runtime.execution.ExecutionState.CANCELED;
import static org.apache.flink.runtime.execution.ExecutionState.FAILED;
import static org.apache.flink.runtime.execution.ExecutionState.FINISHED;
......@@ -63,35 +66,35 @@ public class ExecutionVertex implements Serializable {
private static final long serialVersionUID = 42L;
@SuppressWarnings("unused")
private static final Logger LOG = ExecutionGraph.LOG;
private static final int MAX_DISTINCT_LOCATIONS_TO_CONSIDER = 8;
// --------------------------------------------------------------------------------------------
private final ExecutionJobVertex jobVertex;
private IntermediateResultPartition[] resultPartitions;
private Map<IntermediateResultPartitionID, IntermediateResultPartition> resultPartitions;
private ExecutionEdge[][] inputEdges;
private final int subTaskIndex;
private final List<Execution> priorExecutions;
private final FiniteDuration timeout;
private volatile CoLocationConstraint locationConstraint;
private volatile Execution currentExecution; // this field must never be null
private volatile List<Instance> locationConstraintInstances;
private volatile boolean scheduleLocalOnly;
private StateHandle operatorState;
// --------------------------------------------------------------------------------------------
public ExecutionVertex(ExecutionJobVertex jobVertex, int subTaskIndex,
......@@ -105,11 +108,13 @@ public class ExecutionVertex implements Serializable {
this.jobVertex = jobVertex;
this.subTaskIndex = subTaskIndex;
this.resultPartitions = new IntermediateResultPartition[producedDataSets.length];
for (int i = 0; i < producedDataSets.length; i++) {
IntermediateResultPartition irp = new IntermediateResultPartition(producedDataSets[i], this, subTaskIndex);
this.resultPartitions[i] = irp;
producedDataSets[i].setPartition(subTaskIndex, irp);
this.resultPartitions = new LinkedHashMap<IntermediateResultPartitionID, IntermediateResultPartition>(producedDataSets.length, 1);
for (IntermediateResult result : producedDataSets) {
IntermediateResultPartition irp = new IntermediateResultPartition(result, this, subTaskIndex);
result.setPartition(subTaskIndex, irp);
resultPartitions.put(irp.getPartitionId(), irp);
}
this.inputEdges = new ExecutionEdge[jobVertex.getJobVertex().getInputs().size()][];
......@@ -129,71 +134,71 @@ public class ExecutionVertex implements Serializable {
this.timeout = timeout;
}
// --------------------------------------------------------------------------------------------
// Properties
// --------------------------------------------------------------------------------------------
public JobID getJobId() {
return this.jobVertex.getJobId();
}
public ExecutionJobVertex getJobVertex() {
return jobVertex;
}
public JobVertexID getJobvertexId() {
return this.jobVertex.getJobVertexId();
}
public String getTaskName() {
return this.jobVertex.getJobVertex().getName();
}
public int getTotalNumberOfParallelSubtasks() {
return this.jobVertex.getParallelism();
}
public int getParallelSubtaskIndex() {
return this.subTaskIndex;
}
public int getNumberOfInputs() {
return this.inputEdges.length;
}
public ExecutionEdge[] getInputEdges(int input) {
if (input < 0 || input >= this.inputEdges.length) {
throw new IllegalArgumentException(String.format("Input %d is out of range [0..%d)", input, this.inputEdges.length));
}
return inputEdges[input];
}
public CoLocationConstraint getLocationConstraint() {
return locationConstraint;
}
public Execution getCurrentExecutionAttempt() {
return currentExecution;
}
public ExecutionState getExecutionState() {
return currentExecution.getState();
}
public long getStateTimestamp(ExecutionState state) {
return currentExecution.getStateTimestamp(state);
}
public Throwable getFailureCause() {
return currentExecution.getFailureCause();
}
public SimpleSlot getCurrentAssignedResource() {
return currentExecution.getAssignedResource();
}
public InstanceConnectionInfo getCurrentAssignedResourceLocation() {
return currentExecution.getAssignedResourceLocation();
}
......@@ -213,14 +218,14 @@ public class ExecutionVertex implements Serializable {
// --------------------------------------------------------------------------------------------
// Graph building
// --------------------------------------------------------------------------------------------
public void connectSource(int inputNumber, IntermediateResult source, JobEdge edge, int consumerNumber) {
final DistributionPattern pattern = edge.getDistributionPattern();
final IntermediateResultPartition[] sourcePartitions = source.getPartitions();
ExecutionEdge[] edges;
switch (pattern) {
case POINTWISE:
edges = connectPointwise(sourcePartitions, inputNumber);
......@@ -229,14 +234,14 @@ public class ExecutionVertex implements Serializable {
case ALL_TO_ALL:
edges = connectAllToAll(sourcePartitions, inputNumber);
break;
default:
throw new RuntimeException("Unrecognized distribution pattern.");
}
this.inputEdges[inputNumber] = edges;
// add the consumers to the source
// for now (until the receiver initiated handshake is in place), we need to register the
// edges as the execution graph
......@@ -244,22 +249,22 @@ public class ExecutionVertex implements Serializable {
ee.getSource().addConsumer(ee, consumerNumber);
}
}
private ExecutionEdge[] connectAllToAll(IntermediateResultPartition[] sourcePartitions, int inputNumber) {
ExecutionEdge[] edges = new ExecutionEdge[sourcePartitions.length];
for (int i = 0; i < sourcePartitions.length; i++) {
IntermediateResultPartition irp = sourcePartitions[i];
edges[i] = new ExecutionEdge(irp, this, inputNumber);
}
return edges;
}
private ExecutionEdge[] connectPointwise(IntermediateResultPartition[] sourcePartitions, int inputNumber) {
final int numSources = sourcePartitions.length;
final int parallelism = getTotalNumberOfParallelSubtasks();
// simple case same number of sources as targets
if (numSources == parallelism) {
return new ExecutionEdge[] { new ExecutionEdge(sourcePartitions[subTaskIndex], this, inputNumber) };
......@@ -300,7 +305,7 @@ public class ExecutionVertex implements Serializable {
int start = (int) (subTaskIndex * factor);
int end = (subTaskIndex == getTotalNumberOfParallelSubtasks() - 1) ?
sourcePartitions.length :
sourcePartitions.length :
(int) ((subTaskIndex + 1) * factor);
ExecutionEdge[] edges = new ExecutionEdge[end - start];
......@@ -312,29 +317,29 @@ public class ExecutionVertex implements Serializable {
}
}
}
public void setLocationConstraintHosts(List<Instance> instances) {
this.locationConstraintInstances = instances;
}
public void setScheduleLocalOnly(boolean scheduleLocalOnly) {
if (scheduleLocalOnly && inputEdges != null && inputEdges.length > 0) {
throw new IllegalArgumentException("Strictly local scheduling is only supported for sources.");
}
this.scheduleLocalOnly = scheduleLocalOnly;
}
public boolean isScheduleLocalOnly() {
return scheduleLocalOnly;
}
/**
* Gets the location preferences of this task, determined by the locations of the predecessors from which
* it receives input data.
* If there are more than MAX_DISTINCT_LOCATIONS_TO_CONSIDER different locations of source data, this
* method returns {@code null} to indicate no location preference.
*
*
* @return The preferred locations for this vertex execution, or null, if there is no preference.
*/
public Iterable<Instance> getPreferredLocations() {
......@@ -343,13 +348,12 @@ public class ExecutionVertex implements Serializable {
if (constraintInstances != null && !constraintInstances.isEmpty()) {
return constraintInstances;
}
// otherwise, base the preferred locations on the input connections
if (inputEdges == null) {
return Collections.emptySet();
}
else {
Set<Instance> locations = new HashSet<Instance>();
Set<Instance> inputLocations = new HashSet<Instance>();
......@@ -398,11 +402,11 @@ public class ExecutionVertex implements Serializable {
Execution execution = currentExecution;
ExecutionState state = execution.getState();
if (state == FINISHED || state == CANCELED || state ==FAILED) {
if (state == FINISHED || state == CANCELED || state == FAILED) {
priorExecutions.add(execution);
currentExecution = new Execution(this, execution.getAttemptNumber()+1,
System.currentTimeMillis(), timeout);
CoLocationGroup grp = jobVertex.getCoLocationGroup();
if (grp != null) {
this.locationConstraint = grp.getLocationConstraint(subTaskIndex);
......@@ -436,17 +440,33 @@ public class ExecutionVertex implements Serializable {
}
/**
* Schedules or updates the {@link IntermediateResultPartition} consumer
* tasks of the intermediate result partition with the given index.
* Schedules or updates the consumer tasks of the result partition with the given ID.
*/
void scheduleOrUpdateConsumers(int partitionIndex) {
checkElementIndex(partitionIndex, resultPartitions.length);
void scheduleOrUpdateConsumers(ResultPartitionID partitionId) {
final Execution execution = currentExecution;
IntermediateResultPartition partition = resultPartitions[partitionIndex];
// Abort this request if there was a concurrent reset
if (!partitionId.getProducerId().equals(execution.getAttemptId())) {
return;
}
final IntermediateResultPartition partition = resultPartitions.get(partitionId.getPartitionId());
if (partition == null) {
throw new IllegalStateException("Unknown partition " + partitionId + ".");
}
currentExecution.scheduleOrUpdateConsumers(partition.getConsumers());
if (partition.getIntermediateResult().getResultType().isPipelined()) {
// Schedule or update receivers of this partition
execution.scheduleOrUpdateConsumers(partition.getConsumers());
}
else {
throw new IllegalArgumentException("ScheduleOrUpdateConsumers msg is only valid for" +
"pipelined partitions.");
}
}
/**
* This method cleans fields that are irrelevant for the archived execution attempt.
*/
......@@ -458,22 +478,22 @@ public class ExecutionVertex implements Serializable {
if (!(state == FINISHED || state == CANCELED || state == FAILED)) {
throw new IllegalStateException("Cannot archive ExecutionVertex that is not in a finished state.");
}
// prepare the current execution for archiving
execution.prepareForArchiving();
// prepare previous executions for archiving
for (Execution exec : priorExecutions) {
exec.prepareForArchiving();
}
// clear the unnecessary fields in this class
this.resultPartitions = null;
this.inputEdges = null;
this.locationConstraintInstances = null;
}
public void cachePartitionInfo(PartialPartitionInfo partitionInfo){
public void cachePartitionInfo(PartialInputChannelDeploymentDescriptor partitionInfo){
getCurrentExecutionAttempt().cachePartitionInfo(partitionInfo);
}
......@@ -481,6 +501,25 @@ public class ExecutionVertex implements Serializable {
currentExecution.sendPartitionInfos();
}
/**
* Returns whether to schedule the next batch of receiving tasks.
*/
boolean finishAllBlockingPartitions() {
for (IntermediateResultPartition partition : resultPartitions.values()) {
// Nothing to do for pipelined results
if (partition.getResultType().isPipelined()) {
return false;
}
// It's a blocking partition, mark it as finished and return whether all blocking
// partitions have been produced.
else if (partition.markFinished()) {
return true;
}
}
return false;
}
// --------------------------------------------------------------------------------------------
// Notifications from the Execution Attempt
// --------------------------------------------------------------------------------------------
......@@ -507,20 +546,27 @@ public class ExecutionVertex implements Serializable {
void notifyStateTransition(ExecutionAttemptID executionId, ExecutionState newState, Throwable error) {
getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, executionId, newState, error);
}
TaskDeploymentDescriptor createDeploymentDescriptor(ExecutionAttemptID executionId, SimpleSlot slot) {
/**
* Creates a task deployment descriptor to deploy a subtask to the given target slot.
*/
TaskDeploymentDescriptor createDeploymentDescriptor(
ExecutionAttemptID executionId,
SimpleSlot targetSlot) {
// Produced intermediate results
List<PartitionDeploymentDescriptor> producedPartitions = new ArrayList<PartitionDeploymentDescriptor>(resultPartitions.length);
List<ResultPartitionDeploymentDescriptor> producedPartitions = new ArrayList<ResultPartitionDeploymentDescriptor>(resultPartitions.size());
for (IntermediateResultPartition partition : resultPartitions) {
producedPartitions.add(PartitionDeploymentDescriptor.fromIntermediateResultPartition(partition));
for (IntermediateResultPartition partition : resultPartitions.values()) {
producedPartitions.add(ResultPartitionDeploymentDescriptor.from(partition));
}
// Consumed intermediate results
List<PartitionConsumerDeploymentDescriptor> consumedPartitions = new ArrayList<PartitionConsumerDeploymentDescriptor>();
List<InputGateDeploymentDescriptor> consumedPartitions = new ArrayList<InputGateDeploymentDescriptor>();
for (ExecutionEdge[] edges : inputEdges) {
PartitionInfo[] partitions = PartitionInfo.fromEdges(edges, slot);
InputChannelDeploymentDescriptor[] partitions = InputChannelDeploymentDescriptor
.fromEdges(edges, targetSlot);
// If the produced partition has multiple consumers registered, we
// need to request the one matching our sub task index.
......@@ -531,7 +577,7 @@ public class ExecutionVertex implements Serializable {
IntermediateDataSetID resultId = edges[0].getSource().getIntermediateResult().getId();
consumedPartitions.add(new PartitionConsumerDeploymentDescriptor(resultId, partitions, queueToRequest));
consumedPartitions.add(new InputGateDeploymentDescriptor(resultId, queueToRequest, partitions));
}
List<BlobKey> jarFiles = getExecutionGraph().getRequiredJarFiles();
......@@ -539,7 +585,7 @@ public class ExecutionVertex implements Serializable {
return new TaskDeploymentDescriptor(getJobId(), getJobvertexId(), executionId, getTaskName(),
subTaskIndex, getTotalNumberOfParallelSubtasks(), getExecutionGraph().getJobConfiguration(),
jobVertex.getJobVertex().getConfiguration(), jobVertex.getJobVertex().getInvokableClassName(),
producedPartitions, consumedPartitions, jarFiles, slot.getSlotNumber(), operatorState);
producedPartitions, consumedPartitions, jarFiles, targetSlot.getSlotNumber(), operatorState);
}
// --------------------------------------------------------------------------------------------
......
......@@ -18,8 +18,13 @@
package org.apache.flink.runtime.executiongraph;
import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionType;
import java.util.concurrent.atomic.AtomicInteger;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
public class IntermediateResult {
......@@ -31,20 +36,30 @@ public class IntermediateResult {
private final int numParallelProducers;
private final AtomicInteger numberOfRunningProducers;
private int partitionsAssigned;
private int numConsumers;
private final int connectionIndex;
private final IntermediateResultPartitionType resultType;
private final ResultPartitionType resultType;
public IntermediateResult(
IntermediateDataSetID id,
ExecutionJobVertex producer,
int numParallelProducers,
ResultPartitionType resultType) {
public IntermediateResult(IntermediateDataSetID id, ExecutionJobVertex producer, int numParallelProducers) {
this.id = id;
this.producer = producer;
this.id = checkNotNull(id);
this.producer = checkNotNull(producer);
this.partitions = new IntermediateResultPartition[numParallelProducers];
checkArgument(numParallelProducers >= 1);
this.numParallelProducers = numParallelProducers;
this.numberOfRunningProducers = new AtomicInteger(numParallelProducers);
// we do not set the intermediate result partitions here, because we let them be initialized by
// the execution vertex that produces them
......@@ -52,8 +67,7 @@ public class IntermediateResult {
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.resultType = checkNotNull(resultType);
}
public void setPartition(int partitionNumber, IntermediateResultPartition partition) {
......@@ -85,7 +99,7 @@ public class IntermediateResult {
return partitionsAssigned;
}
public IntermediateResultPartitionType getResultType() {
public ResultPartitionType getResultType() {
return resultType;
}
......@@ -104,4 +118,26 @@ public class IntermediateResult {
public int getConnectionIndex() {
return connectionIndex;
}
void resetForNewExecution() {
this.numberOfRunningProducers.set(numParallelProducers);
}
int decrementNumberOfRunningProducersAndGetRemaining() {
return numberOfRunningProducers.decrementAndGet();
}
boolean isConsumable() {
if (resultType.isPipelined()) {
return true;
}
else {
return numberOfRunningProducers.get() == 0;
}
}
@Override
public String toString() {
return "IntermediateResult " + id.toString();
}
}
......@@ -18,6 +18,7 @@
package org.apache.flink.runtime.executiongraph;
import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
import java.util.ArrayList;
......@@ -59,10 +60,18 @@ public class IntermediateResultPartition {
return partitionId;
}
ResultPartitionType getResultType() {
return totalResult.getResultType();
}
public List<List<ExecutionEdge>> getConsumers() {
return consumers;
}
public boolean isConsumable() {
return totalResult.isConsumable();
}
int addConsumerGroup() {
int pos = consumers.size();
......@@ -78,4 +87,24 @@ public class IntermediateResultPartition {
void addConsumer(ExecutionEdge edge, int consumerNumber) {
consumers.get(consumerNumber).add(edge);
}
boolean markFinished() {
// Sanity check that this is only called on blocking partitions.
if (!getResultType().isBlocking()) {
throw new IllegalStateException("Tried to mark a non-blocking result partition as finished");
}
final int refCnt = totalResult.decrementNumberOfRunningProducersAndGetRemaining();
if (refCnt == 0) {
return true;
}
else if (refCnt < 0) {
throw new IllegalStateException("Decremented number of unfinished producers below 0. "
+ "This is most likely a bug in the execution state/intermediate result "
+ "partition management.");
}
return false;
}
}
......@@ -61,7 +61,7 @@ public class ChannelReaderInputViewIterator<E> implements MutableObjectIterator<
segments, freeMemTarget, accessors, numBlocks);
}
public ChannelReaderInputViewIterator(BlockChannelReader reader, LinkedBlockingQueue<MemorySegment> returnQueue,
public ChannelReaderInputViewIterator(BlockChannelReader<MemorySegment> reader, LinkedBlockingQueue<MemorySegment> returnQueue,
List<MemorySegment> segments, List<MemorySegment> freeMemTarget, TypeSerializer<E> accessors, int numBlocks)
throws IOException
{
......
......@@ -39,7 +39,7 @@ import org.apache.flink.runtime.util.MathUtils;
*/
public class FileChannelInputView extends AbstractPagedInputView {
private final BlockChannelReader reader;
private final BlockChannelReader<MemorySegment> reader;
private final MemoryManager memManager;
......@@ -53,7 +53,7 @@ public class FileChannelInputView extends AbstractPagedInputView {
// --------------------------------------------------------------------------------------------
public FileChannelInputView(BlockChannelReader reader, MemoryManager memManager, List<MemorySegment> memory, int sizeOfLastBlock) throws IOException {
public FileChannelInputView(BlockChannelReader<MemorySegment> reader, MemoryManager memManager, List<MemorySegment> memory, int sizeOfLastBlock) throws IOException {
super(0);
checkNotNull(reader);
......@@ -129,7 +129,7 @@ public class FileChannelInputView extends AbstractPagedInputView {
// get the next segment
numBlocksRemaining--;
return reader.getNextReturnedSegment();
return reader.getNextReturnedBlock();
}
@Override
......
......@@ -35,7 +35,7 @@ import org.apache.flink.runtime.memorymanager.MemoryManager;
*/
public class FileChannelOutputView extends AbstractPagedOutputView {
private final BlockChannelWriter writer; // the writer to the channel
private final BlockChannelWriter<MemorySegment> 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<MemorySegment> memory, int segmentSize) throws IOException {
public FileChannelOutputView(BlockChannelWriter<MemorySegment> writer, MemoryManager memManager, List<MemorySegment> memory, int segmentSize) throws IOException {
super(segmentSize, 0);
checkNotNull(writer);
......@@ -137,7 +137,7 @@ public class FileChannelOutputView extends AbstractPagedOutputView {
if (current != null) {
writeSegment(current, posInSegment);
}
return writer.getNextReturnedSegment();
return writer.getNextReturnedBlock();
}
private void writeSegment(MemorySegment segment, int writePosition) throws IOException {
......
......@@ -40,7 +40,7 @@ import org.apache.flink.runtime.util.MathUtils;
*/
public class SeekableFileChannelInputView extends AbstractPagedInputView {
private BlockChannelReader reader;
private BlockChannelReader<MemorySegment> reader;
private final IOManager ioManager;
......@@ -127,7 +127,7 @@ public class SeekableFileChannelInputView extends AbstractPagedInputView {
}
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 @@ public class SeekableFileChannelInputView extends AbstractPagedInputView {
// get the next segment
numBlocksRemaining--;
return reader.getNextReturnedSegment();
return reader.getNextReturnedBlock();
}
@Override
......
......@@ -42,7 +42,7 @@ public class SpillingBuffer extends AbstractPagedOutputView {
private final MemorySegmentSource memorySource;
private BlockChannelWriter writer;
private BlockChannelWriter<MemorySegment> writer;
private RandomAccessInputView inMemInView;
......@@ -86,7 +86,7 @@ public class SpillingBuffer extends AbstractPagedOutputView {
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 @@ public class SpillingBuffer extends AbstractPagedOutputView {
// spilling
this.writer.writeBlock(current);
this.blockCount++;
return this.writer.getNextReturnedSegment();
return this.writer.getNextReturnedBlock();
}
}
......@@ -116,7 +116,7 @@ public class SpillingBuffer extends AbstractPagedOutputView {
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 class SpillingBuffer extends AbstractPagedOutputView {
this.externalInView.close();
}
final BlockChannelReader reader = this.ioManager.createBlockChannelReader(this.writer.getChannelID());
final BlockChannelReader<MemorySegment> 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 class SpillingBuffer extends AbstractPagedOutputView {
// 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;
......
......@@ -80,7 +80,7 @@ public abstract class AbstractFileIOChannel implements FileIOChannel {
@Override
public abstract void close() throws IOException;
@Override
public void deleteChannel() {
if (!isClosed() || this.fileChannel.isOpen()) {
......@@ -104,4 +104,9 @@ public abstract class AbstractFileIOChannel implements FileIOChannel {
deleteChannel();
}
}
@Override
public FileChannel getNioFileChannel() {
return fileChannel;
}
}
\ No newline at end of file
......@@ -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 @@ import org.apache.flink.core.memory.MemorySegment;
* 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<MemorySegment, ReadRequest> implements BlockChannelReader {
public class AsynchronousBlockReader extends AsynchronousFileIOChannel<MemorySegment, ReadRequest> implements BlockChannelReader<MemorySegment> {
private final LinkedBlockingQueue<MemorySegment> returnSegments;
......@@ -57,7 +57,7 @@ public class AsynchronousBlockReader extends AsynchronousFileIOChannel<MemorySeg
LinkedBlockingQueue<MemorySegment> returnSegments)
throws IOException
{
super(channelID, requestQueue, new QueuingCallback(returnSegments), false);
super(channelID, requestQueue, new QueuingCallback<MemorySegment>(returnSegments), false);
this.returnSegments = returnSegments;
}
......@@ -74,7 +74,12 @@ public class AsynchronousBlockReader extends AsynchronousFileIOChannel<MemorySeg
public void readBlock(MemorySegment segment) throws IOException {
addRequest(new SegmentReadRequest(this, segment));
}
@Override
public void seekToPosition(long position) throws IOException {
requestQueue.add(new SeekRequest(this, position));
}
/**
* 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.
......@@ -87,7 +92,7 @@ public class AsynchronousBlockReader extends AsynchronousFileIOChannel<MemorySeg
* @throws IOException Thrown, if an I/O error occurs in the reader while waiting for the request to return.
*/
@Override
public MemorySegment getNextReturnedSegment() throws IOException {
public MemorySegment getNextReturnedBlock() throws IOException {
try {
while (true) {
final MemorySegment next = this.returnSegments.poll(1000, TimeUnit.MILLISECONDS);
......@@ -115,9 +120,4 @@ public class AsynchronousBlockReader extends AsynchronousFileIOChannel<MemorySeg
public LinkedBlockingQueue<MemorySegment> 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
......@@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit;
import org.apache.flink.core.memory.MemorySegment;
public class AsynchronousBlockWriter extends AsynchronousBlockWriterWithCallback implements BlockChannelWriter {
public class AsynchronousBlockWriter extends AsynchronousBlockWriterWithCallback implements BlockChannelWriter<MemorySegment> {
private final LinkedBlockingQueue<MemorySegment> returnSegments;
......@@ -41,7 +41,7 @@ public class AsynchronousBlockWriter extends AsynchronousBlockWriterWithCallback
LinkedBlockingQueue<MemorySegment> returnSegments)
throws IOException
{
super(channelID, requestQueue, new QueuingCallback(returnSegments));
super(channelID, requestQueue, new QueuingCallback<MemorySegment>(returnSegments));
this.returnSegments = returnSegments;
}
......@@ -58,7 +58,7 @@ public class AsynchronousBlockWriter extends AsynchronousBlockWriterWithCallback
* @throws IOException Thrown, if an I/O error occurs in the writer while waiting for the request to return.
*/
@Override
public MemorySegment getNextReturnedSegment() throws IOException {
public MemorySegment getNextReturnedBlock() throws IOException {
try {
while (true) {
final MemorySegment next = returnSegments.poll(1000, TimeUnit.MILLISECONDS);
......
......@@ -26,7 +26,7 @@ import org.apache.flink.core.memory.MemorySegment;
* An asynchronous implementation of the {@link BlockChannelWriterWithCallback} that queues I/O requests
* and calls a callback once they have been handled.
*/
public class AsynchronousBlockWriterWithCallback extends AsynchronousFileIOChannel<MemorySegment, WriteRequest> implements BlockChannelWriterWithCallback {
public class AsynchronousBlockWriterWithCallback extends AsynchronousFileIOChannel<MemorySegment, WriteRequest> implements BlockChannelWriterWithCallback<MemorySegment> {
/**
* Creates a new asynchronous block writer for the given channel.
......
/*
* 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<Buffer, ReadRequest> implements BufferFileReader {
private final AtomicBoolean hasReachedEndOfFile = new AtomicBoolean();
protected AsynchronousBufferFileReader(ID channelID, RequestQueue<ReadRequest> requestQueue, RequestDoneCallback<Buffer> callback) throws IOException {
super(channelID, requestQueue, callback, false);
}
@Override
public void readInto(Buffer buffer) throws IOException {
addRequest(new BufferReadRequest(this, buffer, hasReachedEndOfFile));
}
@Override
public void seekToPosition(long position) throws IOException {
requestQueue.add(new SeekRequest(this, position));
}
@Override
public boolean hasReachedEndOfFile() {
return hasReachedEndOfFile.get();
}
}
/*
* 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<FileSegment, ReadRequest> implements BufferFileSegmentReader {
private final AtomicBoolean hasReachedEndOfFile = new AtomicBoolean();
protected AsynchronousBufferFileSegmentReader(ID channelID, RequestQueue<ReadRequest> requestQueue, RequestDoneCallback<FileSegment> callback) throws IOException {
super(channelID, requestQueue, callback, false);
}
@Override
public void read() throws IOException {
addRequest(new FileSegmentReadRequest(this, hasReachedEndOfFile));
}
@Override
public void seekTo(long position) throws IOException {
requestQueue.add(new SeekRequest(this, position));
}
@Override
public boolean hasReachedEndOfFile() {
return hasReachedEndOfFile.get();
}
}
/*
* 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<Buffer, WriteRequest> implements BufferFileWriter {
private static final RecyclingCallback CALLBACK = new RecyclingCallback();
protected AsynchronousBufferFileWriter(ID channelID, RequestQueue<WriteRequest> 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 registerAllRequestsProcessedListener(NotificationListener listener) throws IOException {
return super.registerAllRequestsProcessedListener(listener);
}
/**
* Recycles the buffer after the I/O request.
*/
private static class RecyclingCallback implements RequestDoneCallback<Buffer> {
@Override
public void requestSuccessful(Buffer buffer) {
buffer.recycle();
}
@Override
public void requestFailed(Buffer buffer, IOException e) {
buffer.recycle();
}
}
}
......@@ -19,10 +19,13 @@
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.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;
......@@ -37,36 +40,42 @@ import static com.google.common.base.Preconditions.checkNotNull;
* @param <R> The type of request (e.g. <tt>ReadRequest</tt> or <tt>WriteRequest</tt> issued by this access to the I/O threads.
*/
public abstract class AsynchronousFileIOChannel<T, R extends IORequest> 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 listenerLock = 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<R> requestQueue;
/** An atomic integer that counts the number of requests that we still wait for to return. */
protected final AtomicInteger requestsNotReturned = new AtomicInteger(0);
/** Handler for completed requests */
protected final RequestDoneCallback<T> 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 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.
......@@ -79,21 +88,25 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
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 <tt>FileChannel</tt> is closed.
*
* Closes the channel and waits until all pending asynchronous requests are processed. The
* underlying <code>FileChannel</code> is closed even if an exception interrupts the closing.
*
* <p> <strong>Important:</strong> the {@link #isClosed()} method returns <code>true</code>
* immediately after this method has been called even when there are outstanding requests.
*
* @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 +114,7 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
return;
}
this.closed = true;
try {
// wait until as many buffers have been returned as were written
// only then is everything guaranteed to be consistent.
......@@ -136,9 +149,10 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
* <p>
* Even if an exception interrupts the closing, such that not all request are handled,
* the underlying <tt>FileChannel</tt> 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();
......@@ -147,11 +161,11 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
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.
*/
......@@ -160,15 +174,15 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
throw this.exception;
}
}
/**
* Handles a processed <tt>Buffer</tt>. 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) {
......@@ -186,13 +200,26 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
}
}
finally {
// decrement the number of missing buffers. If we are currently closing, notify the waiters
NotificationListener listener = null;
// Decrement the number of outstanding requests. If we are currently closing, notify the
// waiters. If there is a listener, notify her as well.
synchronized (this.closeLock) {
final int num = this.requestsNotReturned.decrementAndGet();
if (this.closed && num == 0) {
this.closeLock.notifyAll();
if (this.requestsNotReturned.decrementAndGet() == 0) {
if (this.closed) {
this.closeLock.notifyAll();
}
synchronized (listenerLock) {
listener = allRequestsProcessedListener;
allRequestsProcessedListener = null;
}
}
}
if (listener != null) {
listener.onNotification();
}
}
}
......@@ -202,14 +229,57 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
// write the current buffer and get the next one
this.requestsNotReturned.incrementAndGet();
if (this.closed || this.requestQueue.isClosed()) {
// if we found ourselves closed after the counter increment,
// decrement the counter again and do not forward the request
this.requestsNotReturned.decrementAndGet();
final NotificationListener listener;
synchronized (listenerLock) {
listener = allRequestsProcessedListener;
allRequestsProcessedListener = null;
}
if (listener != null) {
listener.onNotification();
}
throw new IOException("I/O channel already closed. Could not fulfill: " + request);
}
this.requestQueue.add(request);
}
/**
* Registers a listener to be notified when all outstanding requests have been processed.
*
* <p> New requests can arrive right after the listener got notified. Therefore, it is not safe
* to assume that the number of outstanding requests is still zero after a notification unless
* there was a close right before the listener got called.
*
* <p> Returns <code>true</code>, if the registration was successful. A registration can fail,
* if there are no outstanding requests when trying to register a listener.
*/
protected boolean registerAllRequestsProcessedListener(NotificationListener listener) throws IOException {
checkNotNull(listener);
synchronized (listenerLock) {
if (allRequestsProcessedListener == null) {
// There was a race with the processing of the last outstanding request
if (requestsNotReturned.get() == 0) {
return false;
}
allRequestsProcessedListener = listener;
return true;
}
}
throw new IllegalStateException("Already subscribed.");
}
}
//--------------------------------------------------------------------------------------------
......@@ -218,11 +288,11 @@ public abstract class AsynchronousFileIOChannel<T, R extends IORequest> extends
* Read request that reads an entire memory segment from a block reader.
*/
final class SegmentReadRequest implements ReadRequest {
private final AsynchronousFileIOChannel<MemorySegment, ReadRequest> channel;
private final MemorySegment segment;
protected SegmentReadRequest(AsynchronousFileIOChannel<MemorySegment, ReadRequest> targetChannel, MemorySegment segment) {
this.channel = targetChannel;
this.segment = segment;
......@@ -254,11 +324,11 @@ final class SegmentReadRequest implements ReadRequest {
* Write request that writes an entire memory segment to the block writer.
*/
final class SegmentWriteRequest implements WriteRequest {
private final AsynchronousFileIOChannel<MemorySegment, WriteRequest> channel;
private final MemorySegment segment;
protected SegmentWriteRequest(AsynchronousFileIOChannel<MemorySegment, WriteRequest> targetChannel, MemorySegment segment) {
this.channel = targetChannel;
this.segment = segment;
......@@ -280,6 +350,135 @@ final class SegmentWriteRequest implements WriteRequest {
}
}
final class BufferWriteRequest implements WriteRequest {
private final AsynchronousFileIOChannel<Buffer, WriteRequest> channel;
private final Buffer buffer;
protected BufferWriteRequest(AsynchronousFileIOChannel<Buffer, WriteRequest> targetChannel, Buffer buffer) {
this.channel = checkNotNull(targetChannel);
this.buffer = checkNotNull(buffer);
}
@Override
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<Buffer, ReadRequest> channel;
private final Buffer buffer;
private final AtomicBoolean hasReachedEndOfFile;
protected BufferReadRequest(AsynchronousFileIOChannel<Buffer, ReadRequest> targetChannel, Buffer buffer, AtomicBoolean hasReachedEndOfFile) {
this.channel = targetChannel;
this.buffer = buffer;
this.hasReachedEndOfFile = hasReachedEndOfFile;
}
@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 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();
}
hasReachedEndOfFile.set(fileChannel.size() - fileChannel.position() == 0);
}
else {
hasReachedEndOfFile.set(true);
}
}
@Override
public void requestDone(IOException error) {
channel.handleProcessedBuffer(buffer, error);
}
}
final class FileSegmentReadRequest implements ReadRequest {
private final AsynchronousFileIOChannel<FileSegment, ReadRequest> channel;
private final AtomicBoolean hasReachedEndOfFile;
private FileSegment fileSegment;
protected FileSegmentReadRequest(AsynchronousFileIOChannel<FileSegment, ReadRequest> targetChannel, AtomicBoolean hasReachedEndOfFile) {
this.channel = targetChannel;
this.hasReachedEndOfFile = hasReachedEndOfFile;
}
@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 dataa
fileChannel.position(position + length);
hasReachedEndOfFile.set(fileChannel.size() - fileChannel.position() == 0);
}
else {
hasReachedEndOfFile.set(true);
}
}
@Override
public void requestDone(IOException error) {
channel.handleProcessedBuffer(fileSegment, error);
}
}
/**
* Request that seeks the underlying file channel to the given position.
*/
......
......@@ -21,15 +21,13 @@ package org.apache.flink.runtime.io.disk.iomanager;
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<T> 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.
* <p>
* 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 #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<MemorySegment> getReturnQueue();
/**
* Seeks the underlying file channel to the given position.
*
* @param position The position to seek to.
*/
void seekToPosition(long position) throws IOException;
LinkedBlockingQueue<T> getReturnQueue();
}
\ No newline at end of file
......@@ -21,15 +21,13 @@ package org.apache.flink.runtime.io.disk.iomanager;
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<T> extends BlockChannelWriterWithCallback<T> {
/**
* 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.
* <p>
* 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 #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<MemorySegment> getReturnQueue();
LinkedBlockingQueue<T> getReturnQueue();
}
......@@ -20,16 +20,14 @@ package org.apache.flink.runtime.io.disk.iomanager;
import java.io.IOException;
import org.apache.flink.core.memory.MemorySegment;
public interface BlockChannelWriterWithCallback<T> 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
......@@ -16,18 +16,18 @@
* limitations under the License.
*/
package org.apache.flink.runtime.io.network.partition.queue;
package org.apache.flink.runtime.io.disk.iomanager;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import java.io.IOException;
public class IllegalQueueIteratorRequestException extends IOException {
public interface BufferFileReader extends FileIOChannel {
void readInto(Buffer buffer) throws IOException;
private static final long serialVersionUID = 8381253563445306324L;
void seekToPosition(long position) throws IOException;
public IllegalQueueIteratorRequestException() {
}
boolean hasReachedEndOfFile();
public IllegalQueueIteratorRequestException(String message) {
super(message);
}
}
/*
* 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 hasReachedEndOfFile();
}
......@@ -16,37 +16,23 @@
* limitations under the License.
*/
package org.apache.flink.runtime.io.network.util;
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.util.concurrent.atomic.AtomicInteger;
import java.io.IOException;
public class MockNotificationListener implements NotificationListener {
public interface BufferFileWriter extends BlockChannelWriterWithCallback<Buffer> {
final AtomicInteger numNotifications = new AtomicInteger();
/**
* Returns the number of outstanding requests.
*/
int getNumberOfOutstandingRequests();
@Override
public void onNotification() {
synchronized (numNotifications) {
numNotifications.incrementAndGet();
/**
* Registers a listener, which is notified after all outstanding requests have been processed.
*/
boolean registerAllRequestsProcessedListener(NotificationListener listener) throws IOException;
numNotifications.notifyAll();
}
}
public void waitForNotification() throws InterruptedException {
int current = numNotifications.get();
synchronized (numNotifications) {
while (current == numNotifications.get()) {
numNotifications.wait();
}
}
}
public int getNumberOfNotifications() {
return numNotifications.get();
}
}
\ No newline at end of file
}
......@@ -35,7 +35,7 @@ import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
*/
public class ChannelReaderInputView extends AbstractPagedInputView {
protected final BlockChannelReader reader; // the block reader that reads memory segments
protected final BlockChannelReader<MemorySegment> 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<MemorySegment> memory, boolean waitForFirstBlock)
public ChannelReaderInputView(BlockChannelReader<MemorySegment> reader, List<MemorySegment> memory, boolean waitForFirstBlock)
throws IOException
{
this(reader, memory, -1, waitForFirstBlock);
......@@ -89,7 +89,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<MemorySegment> memory,
public ChannelReaderInputView(BlockChannelReader<MemorySegment> reader, List<MemorySegment> memory,
int numBlocks, boolean waitForFirstBlock)
throws IOException
{
......@@ -117,7 +117,7 @@ public class ChannelReaderInputView extends AbstractPagedInputView {
*
* @throws IOException
*/
ChannelReaderInputView(BlockChannelReader reader, List<MemorySegment> memory,
ChannelReaderInputView(BlockChannelReader<MemorySegment> reader, List<MemorySegment> memory,
int numBlocks, int headerLen, boolean waitForFirstBlock)
throws IOException
{
......@@ -225,7 +225,7 @@ public class ChannelReaderInputView extends AbstractPagedInputView {
}
// 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) {
......
......@@ -61,7 +61,7 @@ public final class ChannelWriterOutputView extends AbstractPagedOutputView {
// --------------------------------------------------------------------------------------------
private final BlockChannelWriter writer; // the writer to the channel
private final BlockChannelWriter<MemorySegment> 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<MemorySegment> memory, int segmentSize) {
public ChannelWriterOutputView(BlockChannelWriter<MemorySegment> writer, List<MemorySegment> memory, int segmentSize) {
super(segmentSize, HEADER_LENGTH);
if (writer == null) {
......@@ -123,7 +123,7 @@ public final class ChannelWriterOutputView extends AbstractPagedOutputView {
* @param writer The writer to write to.
* @param segmentSize The size of the memory segments.
*/
public ChannelWriterOutputView(BlockChannelWriter writer, int segmentSize)
public ChannelWriterOutputView(BlockChannelWriter<MemorySegment> writer, int segmentSize)
{
this(writer, null, segmentSize);
}
......@@ -203,7 +203,7 @@ public final class ChannelWriterOutputView extends AbstractPagedOutputView {
writeSegment(current, posInSegment, false);
}
final MemorySegment next = this.writer.getNextReturnedSegment();
final MemorySegment next = this.writer.getNextReturnedBlock();
this.blockCount++;
return next;
}
......
......@@ -20,6 +20,7 @@ package org.apache.flink.runtime.io.disk.iomanager;
import java.io.File;
import java.io.IOException;
import java.nio.channels.FileChannel;
import java.util.Random;
import org.apache.flink.util.StringUtils;
......@@ -73,6 +74,8 @@ public interface FileIOChannel {
* @throws IOException Thrown, if an error occurred while waiting for pending requests.
*/
public void closeAndDelete() throws IOException;
FileChannel getNioFileChannel();
// --------------------------------------------------------------------------------------------
// --------------------------------------------------------------------------------------------
......
/*
* 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;
}
}
......@@ -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<MemorySegment> memory, int numBlocks,
public HeaderlessChannelReaderInputView(BlockChannelReader<MemorySegment> reader, List<MemorySegment> memory, int numBlocks,
int numBytesInLastBlock, boolean waitForFirstBlock)
throws IOException
{
......@@ -87,7 +87,7 @@ public class HeaderlessChannelReaderInputView extends ChannelReaderInputView
// get the next segment
this.numBlocksRemaining--;
return this.reader.getNextReturnedSegment();
return this.reader.getNextReturnedBlock();
}
......
......@@ -20,6 +20,7 @@ package org.apache.flink.runtime.io.disk.iomanager;
import org.apache.commons.io.FileUtils;
import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......@@ -35,6 +36,21 @@ import java.util.concurrent.LinkedBlockingQueue;
*/
public abstract class IOManager {
public enum IOMode {
SYNC(true), ASYNC(false);
private final boolean isSynchronous;
IOMode(boolean isSynchronous) {
this.isSynchronous = isSynchronous;
}
public boolean isSynchronous() {
return isSynchronous;
}
}
/** Logging */
protected static final Logger LOG = LoggerFactory.getLogger(IOManager.class);
......@@ -190,7 +206,7 @@ public abstract class IOManager {
* @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<MemorySegment> createBlockChannelWriter(FileIOChannel.ID channelID) throws IOException {
return createBlockChannelWriter(channelID, new LinkedBlockingQueue<MemorySegment>());
}
......@@ -203,7 +219,7 @@ public abstract class IOManager {
* @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<MemorySegment> createBlockChannelWriter(FileIOChannel.ID channelID,
LinkedBlockingQueue<MemorySegment> returnQueue) throws IOException;
/**
......@@ -216,7 +232,7 @@ public abstract class IOManager {
* @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<MemorySegment> callback) throws IOException;
public abstract BlockChannelWriterWithCallback<MemorySegment> createBlockChannelWriter(FileIOChannel.ID channelID, RequestDoneCallback<MemorySegment> callback) throws IOException;
/**
* Creates a block channel reader that reads blocks from the given channel. The reader pushed
......@@ -227,7 +243,7 @@ public abstract class IOManager {
* @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<MemorySegment> createBlockChannelReader(FileIOChannel.ID channelID) throws IOException {
return createBlockChannelReader(channelID, new LinkedBlockingQueue<MemorySegment>());
}
......@@ -240,9 +256,15 @@ public abstract class IOManager {
* @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<MemorySegment> createBlockChannelReader(FileIOChannel.ID channelID,
LinkedBlockingQueue<MemorySegment> returnQueue) throws IOException;
public abstract BufferFileWriter createBufferFileWriter(FileIOChannel.ID channelID) throws IOException;
public abstract BufferFileReader createBufferFileReader(FileIOChannel.ID channelID, RequestDoneCallback<Buffer> callback) throws IOException;
public abstract BufferFileSegmentReader createBufferFileSegmentReader(FileIOChannel.ID channelID, RequestDoneCallback<FileSegment> 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
......
......@@ -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;
......@@ -143,7 +144,7 @@ public class IOManagerAsync extends IOManager implements UncaughtExceptionHandle
}
}
finally {
// make sure we all the super implementation in any case and at the last point,
// make sure we call the super implementation in any case and at the last point,
// because this will clean up the I/O directories
super.shutdown();
}
......@@ -182,7 +183,7 @@ public class IOManagerAsync extends IOManager implements UncaughtExceptionHandle
// ------------------------------------------------------------------------
@Override
public BlockChannelWriter createBlockChannelWriter(FileIOChannel.ID channelID,
public BlockChannelWriter<MemorySegment> createBlockChannelWriter(FileIOChannel.ID channelID,
LinkedBlockingQueue<MemorySegment> returnQueue) throws IOException
{
checkState(!isShutdown.get(), "I/O-Manger is shut down.");
......@@ -190,7 +191,7 @@ public class IOManagerAsync extends IOManager implements UncaughtExceptionHandle
}
@Override
public BlockChannelWriterWithCallback createBlockChannelWriter(FileIOChannel.ID channelID, RequestDoneCallback<MemorySegment> callback) throws IOException {
public BlockChannelWriterWithCallback<MemorySegment> createBlockChannelWriter(FileIOChannel.ID channelID, RequestDoneCallback<MemorySegment> callback) throws IOException {
checkState(!isShutdown.get(), "I/O-Manger is shut down.");
return new AsynchronousBlockWriterWithCallback(channelID, this.writers[channelID.getThreadNum()].requestQueue, callback);
}
......@@ -206,13 +207,34 @@ public class IOManagerAsync extends IOManager implements UncaughtExceptionHandle
* @throws IOException Thrown, if the channel for the reader could not be opened.
*/
@Override
public BlockChannelReader createBlockChannelReader(FileIOChannel.ID channelID,
public BlockChannelReader<MemorySegment> createBlockChannelReader(FileIOChannel.ID channelID,
LinkedBlockingQueue<MemorySegment> returnQueue) throws IOException
{
checkState(!isShutdown.get(), "I/O-Manger is shut down.");
return new AsynchronousBlockReader(channelID, this.readers[channelID.getThreadNum()].requestQueue, returnQueue);
}
@Override
public BufferFileWriter createBufferFileWriter(FileIOChannel.ID channelID) throws IOException {
checkState(!isShutdown.get(), "I/O-Manger is shut down.");
return new AsynchronousBufferFileWriter(channelID, writers[channelID.getThreadNum()].requestQueue);
}
@Override
public BufferFileReader createBufferFileReader(FileIOChannel.ID channelID, RequestDoneCallback<Buffer> callback) throws IOException {
checkState(!isShutdown.get(), "I/O-Manger is shut down.");
return new AsynchronousBufferFileReader(channelID, readers[channelID.getThreadNum()].requestQueue, callback);
}
@Override
public BufferFileSegmentReader createBufferFileSegmentReader(FileIOChannel.ID channelID, RequestDoneCallback<FileSegment> callback) throws IOException {
checkState(!isShutdown.get(), "I/O-Manger is shut down.");
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
......
......@@ -21,26 +21,24 @@ package org.apache.flink.runtime.io.disk.iomanager;
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<MemorySegment> {
public class QueuingCallback<T> implements RequestDoneCallback<T> {
private final LinkedBlockingQueue<T> queue;
private final LinkedBlockingQueue<MemorySegment> queue;
public QueuingCallback(LinkedBlockingQueue<MemorySegment> queue) {
public QueuingCallback(LinkedBlockingQueue<T> 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);
}
......
/*
* 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.nio.ByteBuffer;
/**
* A synchronous {@link BufferFileReader} implementation.
*
* <p> This currently bypasses the I/O manager as it is the only synchronous implementation, which
* is currently in use.
*
* TODO Refactor I/O manager setup and refactor this into it
*/
public class SynchronousBufferFileReader extends SynchronousFileIOChannel implements BufferFileReader {
private final ByteBuffer header = ByteBuffer.allocateDirect(8);
private boolean hasReachedEndOfFile;
public SynchronousBufferFileReader(ID channelID, boolean writeEnabled) throws IOException {
super(channelID, writeEnabled);
}
@Override
public void readInto(Buffer buffer) throws IOException {
if (fileChannel.size() - fileChannel.position() > 0) {
// This is the synchronous counter part to the asynchronous buffer read request
// Read header
header.clear();
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();
}
hasReachedEndOfFile = fileChannel.size() - fileChannel.position() == 0;
}
else {
buffer.recycle();
}
}
@Override
public void seekToPosition(long position) throws IOException {
fileChannel.position(position);
}
@Override
public boolean hasReachedEndOfFile() {
return hasReachedEndOfFile;
}
}
......@@ -42,4 +42,4 @@ public abstract class SynchronousFileIOChannel extends AbstractFileIOChannel {
this.fileChannel.close();
}
}
}
\ No newline at end of file
}
......@@ -18,40 +18,34 @@
package org.apache.flink.runtime.io.network;
import java.io.IOException;
import java.io.Serializable;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.runtime.executiongraph.IntermediateResult;
import org.apache.flink.runtime.instance.InstanceConnectionInfo;
import java.io.Serializable;
import java.net.InetSocketAddress;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
/**
* A {@link RemoteAddress} identifies a connection to a remote task manager by
* the socket address and a connection index. This allows multiple connections
* to be distinguished by their connection index.
* <p>
* The connection index is assigned by the {@link IntermediateResult} and
* ensures that it is safe to multiplex multiple data transfers over the same
* physical TCP connection.
* A {@link ConnectionID} identifies a connection to a remote task manager by the socket address and
* a connection index. This allows multiple connections to the same task manager to be distinguished
* by their connection index.
*
* <p> The connection index is assigned by the {@link IntermediateResult} and ensures that it is
* safe to multiplex multiple data transfers over the same physical TCP connection.
*/
public class RemoteAddress implements IOReadableWritable, Serializable {
public class ConnectionID implements Serializable {
private InetSocketAddress address;
private final InetSocketAddress address;
private int connectionIndex;
private final int connectionIndex;
public RemoteAddress(InstanceConnectionInfo connectionInfo, int connectionIndex) {
public ConnectionID(InstanceConnectionInfo connectionInfo, int connectionIndex) {
this(new InetSocketAddress(connectionInfo.address(), connectionInfo.dataPort()), connectionIndex);
}
public RemoteAddress(InetSocketAddress address, int connectionIndex) {
public ConnectionID(InetSocketAddress address, int connectionIndex) {
this.address = checkNotNull(address);
checkArgument(connectionIndex >= 0);
this.connectionIndex = connectionIndex;
......@@ -72,11 +66,11 @@ public class RemoteAddress implements IOReadableWritable, Serializable {
@Override
public boolean equals(Object other) {
if (other.getClass() != RemoteAddress.class) {
if (other.getClass() != ConnectionID.class) {
return false;
}
final RemoteAddress ra = (RemoteAddress) other;
final ConnectionID ra = (ConnectionID) other;
if (!ra.getAddress().equals(address) || ra.getConnectionIndex() != connectionIndex) {
return false;
}
......@@ -88,35 +82,4 @@ public class RemoteAddress implements IOReadableWritable, Serializable {
public String toString() {
return address + " [" + connectionIndex + "]";
}
// ------------------------------------------------------------------------
// Serialization
// ------------------------------------------------------------------------
public RemoteAddress() {
this.address = null;
this.connectionIndex = -1;
}
@Override
public void write(final DataOutputView out) throws IOException {
final InetAddress ia = address.getAddress();
out.writeInt(ia.getAddress().length);
out.write(ia.getAddress());
out.writeInt(address.getPort());
out.writeInt(connectionIndex);
}
@Override
public void read(final DataInputView in) throws IOException {
final byte[] addressBytes = new byte[in.readInt()];
in.readFully(addressBytes);
final InetAddress ia = InetAddress.getByAddress(addressBytes);
int port = in.readInt();
address = new InetSocketAddress(ia, port);
connectionIndex = in.readInt();
}
}
......@@ -18,8 +18,9 @@
package org.apache.flink.runtime.io.network;
import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
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,20 +30,20 @@ import java.io.IOException;
*/
public interface ConnectionManager {
void start(IntermediateResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher) throws IOException;
void start(ResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher, NetworkBufferPool networkbufferPool) throws IOException;
/**
* Creates a {@link PartitionRequestClient} instance for the given {@link RemoteAddress}.
* Creates a {@link PartitionRequestClient} instance for the given {@link ConnectionID}.
*/
PartitionRequestClient createPartitionRequestClient(RemoteAddress remoteAddress) throws IOException, InterruptedException;
PartitionRequestClient createPartitionRequestClient(ConnectionID connectionId) throws IOException, InterruptedException;
/**
* Closes opened ChannelConnections in case of a resource release
* @param remoteAddress
*/
void closeOpenChannelConnections(RemoteAddress remoteAddress);
void closeOpenChannelConnections(ConnectionID connectionId);
int getNumberOfActiveConnections();
void shutdown() throws IOException;
}
......@@ -18,8 +18,9 @@
package org.apache.flink.runtime.io.network;
import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
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,16 +31,16 @@ import java.io.IOException;
public class LocalConnectionManager implements ConnectionManager {
@Override
public void start(IntermediateResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher) throws IOException {
public void start(ResultPartitionProvider partitionProvider, TaskEventDispatcher taskEventDispatcher, NetworkBufferPool networkbufferPool) throws IOException {
}
@Override
public PartitionRequestClient createPartitionRequestClient(RemoteAddress remoteAddress) throws IOException {
public PartitionRequestClient createPartitionRequestClient(ConnectionID connectionId) throws IOException {
return null;
}
@Override
public void closeOpenChannelConnections(RemoteAddress remoteAddress) {}
public void closeOpenChannelConnections(ConnectionID connectionId) {}
@Override
public int getNumberOfActiveConnections() {
......
......@@ -21,13 +21,14 @@ package org.apache.flink.runtime.io.network;
import akka.actor.ActorRef;
import akka.util.Timeout;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.io.network.api.writer.BufferWriter;
import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode;
import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
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.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.io.network.partition.consumer.SingleInputGate;
import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
import org.apache.flink.runtime.taskmanager.Task;
......@@ -54,7 +55,7 @@ public class NetworkEnvironment {
private final FiniteDuration jobManagerTimeout;
private final IntermediateResultPartitionManager partitionManager;
private final ResultPartitionManager partitionManager;
private final TaskEventDispatcher taskEventDispatcher;
......@@ -62,6 +63,8 @@ public class NetworkEnvironment {
private final ConnectionManager connectionManager;
private final NetworkEnvironmentConfiguration configuration;
private boolean isShutdown;
/**
......@@ -74,8 +77,9 @@ public class NetworkEnvironment {
this.jobManager = checkNotNull(jobManager);
this.jobManagerTimeout = checkNotNull(jobManagerTimeout);
this.partitionManager = new IntermediateResultPartitionManager();
this.partitionManager = new ResultPartitionManager();
this.taskEventDispatcher = new TaskEventDispatcher();
this.configuration = checkNotNull(config);
// --------------------------------------------------------------------
// Network buffers
......@@ -95,7 +99,7 @@ public class NetworkEnvironment {
connectionManager = nettyConfig.isDefined() ? new NettyConnectionManager(nettyConfig.get()) : new LocalConnectionManager();
try {
connectionManager.start(partitionManager, taskEventDispatcher);
connectionManager.start(partitionManager, taskEventDispatcher, networkBufferPool);
}
catch (Throwable t) {
throw new IOException("Failed to instantiate network connection manager: " + t.getMessage(), t);
......@@ -115,30 +119,29 @@ public class NetworkEnvironment {
}
public void registerTask(Task task) throws IOException {
final ExecutionAttemptID executionId = task.getExecutionId();
final IntermediateResultPartition[] producedPartitions = task.getProducedPartitions();
final BufferWriter[] writers = task.getWriters();
final ResultPartition[] producedPartitions = task.getProducedPartitions();
final ResultPartitionWriter[] writers = task.getWriters();
if (writers.length != producedPartitions.length) {
throw new IllegalStateException("Unequal number of writers and partitions.");
}
for (int i = 0; i < producedPartitions.length; i++) {
final IntermediateResultPartition partition = producedPartitions[i];
final BufferWriter writer = writers[i];
final ResultPartition partition = producedPartitions[i];
final ResultPartitionWriter 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) {
......@@ -150,7 +153,7 @@ public class NetworkEnvironment {
}
// Register writer with task event dispatcher
taskEventDispatcher.registerWriterForIncomingTaskEvents(executionId, writer.getPartitionId(), writer);
taskEventDispatcher.registerWriterForIncomingTaskEvents(writer.getPartitionId(), writer);
}
// Setup the buffer pool for each buffer reader
......@@ -165,7 +168,7 @@ public class NetworkEnvironment {
}
catch (Throwable t) {
if (bufferPool != null) {
bufferPool.destroy();
bufferPool.lazyDestroy();
}
if (t instanceof IOException) {
......@@ -185,10 +188,16 @@ public class NetworkEnvironment {
final ExecutionAttemptID executionId = task.getExecutionId();
if (task.isCanceledOrFailed()) {
partitionManager.failIntermediateResultPartitions(executionId);
partitionManager.releasePartitionsProducedBy(executionId);
}
taskEventDispatcher.unregisterWriters(executionId);
ResultPartitionWriter[] writers = task.getWriters();
if (writers != null) {
for (ResultPartitionWriter writer : task.getWriters()) {
taskEventDispatcher.unregisterWriter(writer);
}
}
final SingleInputGate[] inputGates = task.getInputGates();
......@@ -206,7 +215,7 @@ public class NetworkEnvironment {
}
}
public IntermediateResultPartitionManager getPartitionManager() {
public ResultPartitionManager getPartitionManager() {
return partitionManager;
}
......@@ -222,6 +231,10 @@ public class NetworkEnvironment {
return networkBufferPool;
}
public IOMode getDefaultIOMode() {
return configuration.ioMode();
}
public boolean hasReleasedAllResources() {
String msg = String.format("Network buffer pool: %d missing memory segments. %d registered buffer pools. Connection manager: %d active connections. Task event dispatcher: %d registered writers.",
networkBufferPool.getTotalNumberOfMemorySegments() - networkBufferPool.getNumberOfAvailableMemorySegments(), networkBufferPool.getNumberOfRegisteredBufferPools(), connectionManager.getNumberOfActiveConnections(), taskEventDispatcher.getNumberOfRegisteredWriters());
......
......@@ -18,59 +18,49 @@
package org.apache.flink.runtime.io.network;
import com.google.common.collect.HashBasedTable;
import com.google.common.collect.Table;
import com.google.common.collect.Maps;
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.io.network.api.writer.ResultPartitionWriter;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
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.util.event.EventListener;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
/**
* The task event dispatcher dispatches events flowing backwards from a consumer
* to a producer. It only supports programs, where the producer and consumer
* are running at the same time.
* <p>
* The publish method is either called from the local input channel or the
* network I/O thread.
* The task event dispatcher dispatches events flowing backwards from a consuming task to the task
* producing the consumed result.
*
* <p> Backwards events only work for tasks, which produce pipelined results, where both the
* producing and consuming task are running at the same time.
*/
public class TaskEventDispatcher {
Table<ExecutionAttemptID, IntermediateResultPartitionID, BufferWriter> registeredWriters = HashBasedTable.create();
private final Map<ResultPartitionID, ResultPartitionWriter> registeredWriters = Maps.newHashMap();
public void registerWriterForIncomingTaskEvents(ExecutionAttemptID executionId, IntermediateResultPartitionID partitionId, BufferWriter listener) {
public void registerWriterForIncomingTaskEvents(ResultPartitionID partitionId, ResultPartitionWriter writer) {
synchronized (registeredWriters) {
if (registeredWriters.put(executionId, partitionId, listener) != null) {
throw new IllegalStateException("Event dispatcher already contains buffer writer.");
if (registeredWriters.put(partitionId, writer) != null) {
throw new IllegalStateException("Already registered at task event dispatcher.");
}
}
}
public void unregisterWriters(ExecutionAttemptID executionId) {
public void unregisterWriter(ResultPartitionWriter writer) {
synchronized (registeredWriters) {
List<IntermediateResultPartitionID> writersToUnregister = new ArrayList<IntermediateResultPartitionID>();
for (IntermediateResultPartitionID partitionId : registeredWriters.row(executionId).keySet()) {
writersToUnregister.add(partitionId);
}
for(IntermediateResultPartitionID partitionId : writersToUnregister) {
registeredWriters.remove(executionId, partitionId);
}
registeredWriters.remove(writer.getPartitionId());
}
}
/**
* Publishes the event to the registered {@link EventListener} instance.
* Publishes the event to the registered {@link ResultPartitionWriter} instances.
* <p>
* This method is either called from a local input channel or the network
* I/O thread on behalf of a remote input channel.
* This method is either called directly from a {@link LocalInputChannel} or the network I/O
* thread on behalf of a {@link RemoteInputChannel}.
*/
public boolean publish(ExecutionAttemptID executionId, IntermediateResultPartitionID partitionId, TaskEvent event) {
EventListener<TaskEvent> listener = registeredWriters.get(executionId, partitionId);
public boolean publish(ResultPartitionID partitionId, TaskEvent event) {
EventListener<TaskEvent> listener = registeredWriters.get(partitionId);
if (listener != null) {
listener.onEvent(event);
......@@ -80,6 +70,9 @@ public class TaskEventDispatcher {
return false;
}
/**
* Returns the number of currently registered writers.
*/
int getNumberOfRegisteredWriters() {
synchronized (registeredWriters) {
return registeredWriters.size();
......
......@@ -21,10 +21,10 @@ package org.apache.flink.runtime.io.network.api.reader;
import org.apache.flink.core.io.IOReadableWritable;
import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult;
import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer;
import org.apache.flink.runtime.io.network.buffer.Buffer;
import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.io.network.serialization.SpillingAdaptiveSpanningRecordDeserializer;
import java.io.IOException;
......@@ -43,6 +43,7 @@ abstract class AbstractRecordReader<T extends IOReadableWritable> extends Abstra
private boolean isFinished;
@SuppressWarnings("unchecked")
protected AbstractRecordReader(InputGate inputGate) {
super(inputGate);
......
......@@ -16,7 +16,7 @@
* limitations under the License.
*/
package org.apache.flink.runtime.io.network.serialization;
package org.apache.flink.runtime.io.network.api.serialization;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.GlobalConfiguration;
......
......@@ -31,7 +31,7 @@ import static org.apache.flink.runtime.io.network.api.serialization.RecordSerial
/**
* A record-oriented runtime result writer.
* <p>
* The RecordWriter wraps the runtime's {@link BufferWriter} and takes care of
* The RecordWriter wraps the runtime's {@link ResultPartitionWriter} and takes care of
* serializing records into buffers.
* <p>
* <strong>Important</strong>: it is necessary to call {@link #flush()} after
......@@ -43,7 +43,7 @@ import static org.apache.flink.runtime.io.network.api.serialization.RecordSerial
*/
public class RecordWriter<T extends IOReadableWritable> {
protected final BufferWriter writer;
protected final ResultPartitionWriter writer;
private final ChannelSelector<T> channelSelector;
......@@ -52,11 +52,12 @@ public class RecordWriter<T extends IOReadableWritable> {
/** {@link RecordSerializer} per outgoing channel */
private final RecordSerializer<T>[] serializers;
public RecordWriter(BufferWriter writer) {
public RecordWriter(ResultPartitionWriter writer) {
this(writer, new RoundRobinChannelSelector<T>());
}
public RecordWriter(BufferWriter writer, ChannelSelector<T> channelSelector) {
@SuppressWarnings("unchecked")
public RecordWriter(ResultPartitionWriter writer, ChannelSelector<T> channelSelector) {
this.writer = writer;
this.channelSelector = channelSelector;
......@@ -73,10 +74,6 @@ public class RecordWriter<T extends IOReadableWritable> {
}
}
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
......
......@@ -21,32 +21,32 @@ package org.apache.flink.runtime.io.network.api.writer;
import org.apache.flink.runtime.event.task.AbstractEvent;
import org.apache.flink.runtime.event.task.TaskEvent;
import org.apache.flink.runtime.io.network.api.EndOfSuperstepEvent;
import org.apache.flink.runtime.io.network.api.TaskEventHandler;
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.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.util.event.EventListener;
import org.apache.flink.runtime.io.network.api.TaskEventHandler;
import java.io.IOException;
/**
* A buffer-oriented runtime result writer.
* <p>
* The {@link BufferWriter} is the runtime API for producing results. It
* The {@link ResultPartitionWriter} is the runtime API for producing results. It
* supports two kinds of data to be sent: buffers and events.
* <p>
* <strong>Important</strong>: When working directly with this API, it is
* necessary to call {@link #finish()} after all data has been produced.
*/
public final class BufferWriter implements EventListener<TaskEvent> {
public final class ResultPartitionWriter implements EventListener<TaskEvent> {
private final IntermediateResultPartition partition;
private final ResultPartition partition;
private final TaskEventHandler taskEventHandler = new TaskEventHandler();
public BufferWriter(IntermediateResultPartition partition) {
public ResultPartitionWriter(ResultPartition partition) {
this.partition = partition;
}
......@@ -54,7 +54,7 @@ public final class BufferWriter implements EventListener<TaskEvent> {
// Attributes
// ------------------------------------------------------------------------
public IntermediateResultPartitionID getPartitionId() {
public ResultPartitionID getPartitionId() {
return partition.getPartitionId();
}
......@@ -63,7 +63,7 @@ public final class BufferWriter implements EventListener<TaskEvent> {
}
public int getNumberOfOutputChannels() {
return partition.getNumberOfQueues();
return partition.getNumberOfSubpartitions();
}
// ------------------------------------------------------------------------
......@@ -79,14 +79,14 @@ public final class BufferWriter implements EventListener<TaskEvent> {
}
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 final class BufferWriter implements EventListener<TaskEvent> {
partition.finish();
}
public boolean isFinished() {
return partition.isFinished();
}
// ------------------------------------------------------------------------
// Event handling
// ------------------------------------------------------------------------
......
......@@ -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 class Buffer {
return isBuffer;
}
public void tagAsEvent() {
synchronized (recycleLock) {
ensureNotRecycled();
}
isBuffer = false;
}
public MemorySegment getMemorySegment() {
synchronized (recycleLock) {
ensureNotRecycled();
......@@ -84,8 +92,6 @@ public class Buffer {
public int getSize() {
synchronized (recycleLock) {
ensureNotRecycled();
return currentSize;
}
}
......@@ -103,8 +109,6 @@ public class Buffer {
public void recycle() {
synchronized (recycleLock) {
ensureNotRecycled();
if (--referenceCount == 0) {
recycler.recycle(memorySegment);
}
......
......@@ -24,7 +24,7 @@ public interface BufferPool extends BufferProvider, BufferRecycler {
void setBufferPoolOwner(BufferPoolOwner owner);
void destroy() throws IOException;
void lazyDestroy();
@Override
boolean isDestroyed();
......
......@@ -22,6 +22,6 @@ import java.io.IOException;
public interface BufferPoolOwner {
void recycleBuffers(int numBuffersToRecycle) throws IOException;
void releaseMemory(int numBuffersToRecycle) throws IOException;
}
......@@ -52,4 +52,6 @@ public interface BufferProvider {
boolean isDestroyed();
int getMemorySegmentSize();
}
......@@ -82,6 +82,11 @@ class LocalBufferPool implements BufferPool {
}
}
@Override
public int getMemorySegmentSize() {
return networkBufferPool.getMemorySegmentSize();
}
@Override
public int getNumberOfRequiredMemorySegments() {
return numberOfRequiredMemorySegments;
......@@ -124,15 +129,17 @@ class LocalBufferPool implements BufferPool {
return requestBuffer(true);
}
private Buffer requestBuffer(boolean isBlocking) throws InterruptedException {
private Buffer requestBuffer(boolean isBlocking) throws InterruptedException, IOException {
synchronized (availableMemorySegments) {
if (isDestroyed) {
return null;
}
returnExcessMemorySegments();
boolean askToRecycle = owner != null;
while (availableMemorySegments.isEmpty()) {
if (isDestroyed) {
return null;
}
if (numberOfRequestedMemorySegments < currentPoolSize) {
final MemorySegment segment = networkBufferPool.requestMemorySegment();
......@@ -144,6 +151,10 @@ class LocalBufferPool implements BufferPool {
}
}
if (askToRecycle) {
owner.releaseMemory(1);
}
if (isBlocking) {
availableMemorySegments.wait(2000);
}
......@@ -186,7 +197,7 @@ class LocalBufferPool implements BufferPool {
* 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 +241,7 @@ class LocalBufferPool implements BufferPool {
// 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);
}
}
}
......
......@@ -185,14 +185,18 @@ public class NetworkBufferPool implements BufferPoolFactory {
}
@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);
}
}
}
}
......
......@@ -63,7 +63,12 @@ public class NettyConfig {
private final Configuration config; // optional configuration
public NettyConfig(InetAddress serverAddress, int serverPort, int memorySegmentSize, Configuration config) {
public NettyConfig(
InetAddress serverAddress,
int serverPort,
int memorySegmentSize,
Configuration config) {
this.serverAddress = checkNotNull(serverAddress);
checkArgument(serverPort > 0 && serverPort <= 65536, "Invalid port number.");
......@@ -93,42 +98,42 @@ public class NettyConfig {
// Setters
// ------------------------------------------------------------------------
NettyConfig setServerConnectBacklog(int connectBacklog) {
public NettyConfig setServerConnectBacklog(int connectBacklog) {
checkArgument(connectBacklog >= 0);
config.setInteger(CONNECT_BACKLOG, connectBacklog);
return this;
}
NettyConfig setServerNumThreads(int numThreads) {
public NettyConfig setServerNumThreads(int numThreads) {
checkArgument(numThreads >= 0);
config.setInteger(NUM_THREADS_SERVER, numThreads);
return this;
}
NettyConfig setClientNumThreads(int numThreads) {
public NettyConfig setClientNumThreads(int numThreads) {
checkArgument(numThreads >= 0);
config.setInteger(NUM_THREADS_CLIENT, numThreads);
return this;
}
NettyConfig setClientConnectTimeoutSeconds(int connectTimeoutSeconds) {
public NettyConfig setClientConnectTimeoutSeconds(int connectTimeoutSeconds) {
checkArgument(connectTimeoutSeconds >= 0);
config.setInteger(CLIENT_CONNECT_TIMEOUT_SECONDS, connectTimeoutSeconds);
return this;
}
NettyConfig setSendAndReceiveBufferSize(int bufferSize) {
public NettyConfig setSendAndReceiveBufferSize(int bufferSize) {
checkArgument(bufferSize >= 0);
config.setInteger(SEND_RECEIVE_BUFFER_SIZE, bufferSize);
return this;
}
NettyConfig setTransportType(String transport) {
public NettyConfig setTransportType(String transport) {
if (transport.equals("nio") || transport.equals("epoll") || transport.equals("auto")) {
config.setString(TRANSPORT_TYPE, transport);
}
......@@ -143,32 +148,32 @@ public class NettyConfig {
// Getters
// ------------------------------------------------------------------------
int getServerConnectBacklog() {
public int getServerConnectBacklog() {
// default: 0 => Netty's default
return config.getInteger(CONNECT_BACKLOG, 0);
}
int getServerNumThreads() {
public int getServerNumThreads() {
// default: 0 => Netty's default: 2 * #cores
return config.getInteger(NUM_THREADS_SERVER, 0);
}
int getClientNumThreads() {
public int getClientNumThreads() {
// default: 0 => Netty's default: 2 * #cores
return config.getInteger(NUM_THREADS_CLIENT, 0);
}
int getClientConnectTimeoutSeconds() {
public int getClientConnectTimeoutSeconds() {
// default: 120s = 2min
return config.getInteger(CLIENT_CONNECT_TIMEOUT_SECONDS, 120);
}
int getSendAndReceiveBufferSize() {
public int getSendAndReceiveBufferSize() {
// default: 0 => Netty's default
return config.getInteger(SEND_RECEIVE_BUFFER_SIZE, 0);
}
TransportType getTransportType() {
public TransportType getTransportType() {
String transport = config.getString(TRANSPORT_TYPE, "nio");
if (transport.equals("nio")) {
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册