提交 4cd4a134 编写于 作者: U uce 提交者: StephanEwen

Replace custom Java NIO TCP/IP code with Netty 4 library

上级 2db78a8d
...@@ -35,7 +35,6 @@ public final class ConfigConstants { ...@@ -35,7 +35,6 @@ public final class ConfigConstants {
*/ */
public static final String PARALLELIZATION_MAX_INTRA_NODE_DEGREE_KEY = "parallelization.intra-node.default"; public static final String PARALLELIZATION_MAX_INTRA_NODE_DEGREE_KEY = "parallelization.intra-node.default";
// -------------------------------- Runtime ------------------------------- // -------------------------------- Runtime -------------------------------
/** /**
...@@ -99,6 +98,26 @@ public final class ConfigConstants { ...@@ -99,6 +98,26 @@ public final class ConfigConstants {
*/ */
public static final String TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY = "taskmanager.network.bufferSizeInBytes"; public static final String TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY = "taskmanager.network.bufferSizeInBytes";
/**
* The number of incoming connection threads used in NettyConnectionManager for the ServerBootstrap.
*/
public static final String TASK_MANAGER_NETTY_NUM_IN_THREADS_KEY = "taskmanager.netty.numInThreads";
/**
* The number of outgoing connection threads used in NettyConnectionManager for the Bootstrap.
*/
public static final String TASK_MANAGER_NETTY_NUM_OUT_THREADS_KEY = "taskmanager.netty.numOutThreads";
/**
* The low water mark used in NettyConnectionManager for the Bootstrap.
*/
public static final String TASK_MANAGER_NETTY_LOW_WATER_MARK = "taskmanager.netty.lowWaterMark";
/**
* The high water mark used in NettyConnectionManager for the Bootstrap.
*/
public static final String TASK_MANAGER_NETTY_HIGH_WATER_MARK = "taskmanager.netty.highWaterMark";
/** /**
* Parameter for the interval in which the RaskManager sends the periodic heart beat messages * Parameter for the interval in which the RaskManager sends the periodic heart beat messages
* to the JobManager (in msecs). * to the JobManager (in msecs).
...@@ -135,7 +154,6 @@ public final class ConfigConstants { ...@@ -135,7 +154,6 @@ public final class ConfigConstants {
*/ */
public static final String JOBCLIENT_POLLING_INTERVAL_KEY = "jobclient.polling.interval"; public static final String JOBCLIENT_POLLING_INTERVAL_KEY = "jobclient.polling.interval";
// ------------------------ Hadoop Configuration ------------------------ // ------------------------ Hadoop Configuration ------------------------
/** /**
...@@ -153,7 +171,6 @@ public final class ConfigConstants { ...@@ -153,7 +171,6 @@ public final class ConfigConstants {
*/ */
public static final String PATH_HADOOP_CONFIG = "fs.hdfs.hadoopconf"; public static final String PATH_HADOOP_CONFIG = "fs.hdfs.hadoopconf";
// ------------------------ File System Bahavior ------------------------ // ------------------------ File System Bahavior ------------------------
/** /**
...@@ -252,10 +269,6 @@ public final class ConfigConstants { ...@@ -252,10 +269,6 @@ public final class ConfigConstants {
public static final String STRATOSPHERE_JVM_OPTIONS = "env.java.opts"; public static final String STRATOSPHERE_JVM_OPTIONS = "env.java.opts";
// ------------------------------------------------------------------------ // ------------------------------------------------------------------------
// Default Values // Default Values
// ------------------------------------------------------------------------ // ------------------------------------------------------------------------
...@@ -319,6 +332,30 @@ public final class ConfigConstants { ...@@ -319,6 +332,30 @@ public final class ConfigConstants {
*/ */
public static final int DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE = 32768; public static final int DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE = 32768;
/**
* Default number of incoming connection threads used in NettyConnectionManager for the ServerBootstrap. If set
* to -1, NettyConnectionManager will pick a reasonable default depending on the number of cores of the machine.
*/
public static final int DEFAULT_TASK_MANAGER_NETTY_NUM_IN_THREADS = -1;
/**
* Default number of outgoing connection threads used in NettyConnectionManager for the Bootstrap. If set
* to -1, NettyConnectionManager will pick a reasonable default depending on the number of cores of the machine.
*/
public static final int DEFAULT_TASK_MANAGER_NETTY_NUM_OUT_THREADS = -1;
/**
* Default low water mark used in NettyConnectionManager for the Bootstrap. If set to -1, NettyConnectionManager
* will use half of the network buffer size as the low water mark.
*/
public static final int DEFAULT_TASK_MANAGER_NETTY_LOW_WATER_MARK = -1;
/**
* Default high water mark used in NettyConnectionManager for the Bootstrap. If set to -1, NettyConnectionManager
* will use the network buffer size as the high water mark.
*/
public static final int DEFAULT_TASK_MANAGER_NETTY_HIGH_WATER_MARK = -1;
/** /**
* The default interval for TaskManager heart beats (2000 msecs). * The default interval for TaskManager heart beats (2000 msecs).
*/ */
...@@ -452,7 +489,6 @@ public final class ConfigConstants { ...@@ -452,7 +489,6 @@ public final class ConfigConstants {
*/ */
public static final int DEFAULT_DEFAULT_INSTANCE_TYPE_INDEX = 1; public static final int DEFAULT_DEFAULT_INSTANCE_TYPE_INDEX = 1;
// ------------------------------------------------------------------------ // ------------------------------------------------------------------------
/** /**
......
...@@ -53,6 +53,12 @@ ...@@ -53,6 +53,12 @@
<version>1.2.1</version> <version>1.2.1</version>
</dependency> </dependency>
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
<version>4.0.19.Final</version>
</dependency>
<dependency> <dependency>
<groupId>eu.stratosphere</groupId> <groupId>eu.stratosphere</groupId>
<artifactId>stratosphere-java</artifactId> <artifactId>stratosphere-java</artifactId>
......
...@@ -20,6 +20,7 @@ import java.nio.ByteBuffer; ...@@ -20,6 +20,7 @@ import java.nio.ByteBuffer;
import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.IOReadableWritable;
import eu.stratosphere.util.StringUtils; import eu.stratosphere.util.StringUtils;
import io.netty.buffer.ByteBuf;
/** /**
* A statistically unique identification number. * A statistically unique identification number.
...@@ -167,6 +168,11 @@ public class AbstractID implements IOReadableWritable { ...@@ -167,6 +168,11 @@ public class AbstractID implements IOReadableWritable {
buffer.putLong(this.upperPart); buffer.putLong(this.upperPart);
} }
public void writeTo(ByteBuf buf) {
buf.writeLong(this.lowerPart);
buf.writeLong(this.upperPart);
}
@Override @Override
public String toString() { public String toString() {
final byte[] ba = new byte[SIZE]; final byte[] ba = new byte[SIZE];
......
...@@ -14,9 +14,6 @@ ...@@ -14,9 +14,6 @@
**********************************************************************************************************************/ **********************************************************************************************************************/
package eu.stratosphere.nephele.execution; package eu.stratosphere.nephele.execution;
import eu.stratosphere.pact.runtime.task.chaining.ExceptionInChainedStubException;
/** /**
* Thrown to trigger a canceling of the executing task. Intended to cause a cancelled status, rather than a failed status. * Thrown to trigger a canceling of the executing task. Intended to cause a cancelled status, rather than a failed status.
*/ */
......
...@@ -16,17 +16,21 @@ package eu.stratosphere.nephele.execution; ...@@ -16,17 +16,21 @@ package eu.stratosphere.nephele.execution;
import eu.stratosphere.configuration.Configuration; import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.core.fs.Path; import eu.stratosphere.core.fs.Path;
import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.IOReadableWritable;
import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.runtime.io.gates.GateID;
import eu.stratosphere.runtime.io.gates.InputGate;
import eu.stratosphere.runtime.io.gates.OutputGate;
import eu.stratosphere.nephele.jobgraph.JobGraph; import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.nephele.protocols.AccumulatorProtocol; import eu.stratosphere.nephele.protocols.AccumulatorProtocol;
import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.iomanager.IOManager;
import eu.stratosphere.nephele.services.memorymanager.MemoryManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
import eu.stratosphere.nephele.template.InputSplitProvider; import eu.stratosphere.nephele.template.InputSplitProvider;
import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.runtime.io.gates.GateID;
import eu.stratosphere.runtime.io.gates.InputGate;
import eu.stratosphere.runtime.io.gates.OutputGate;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.FutureTask;
/** /**
* The user code of every Nephele task runs inside an <code>Environment</code> object. The environment provides * The user code of every Nephele task runs inside an <code>Environment</code> object. The environment provides
...@@ -74,16 +78,14 @@ public interface Environment { ...@@ -74,16 +78,14 @@ public interface Environment {
/** /**
* Sends a notification that objects that a new user thread has been started to the execution observer. * Sends a notification that objects that a new user thread has been started to the execution observer.
* *
* @param userThread * @param userThread the user thread which has been started
* the user thread which has been started
*/ */
void userThreadStarted(Thread userThread); void userThreadStarted(Thread userThread);
/** /**
* Sends a notification that a user thread has finished to the execution observer. * Sends a notification that a user thread has finished to the execution observer.
* *
* @param userThread * @param userThread the user thread which has finished
* the user thread which has finished
*/ */
void userThreadFinished(Thread userThread); void userThreadFinished(Thread userThread);
...@@ -193,8 +195,7 @@ public interface Environment { ...@@ -193,8 +195,7 @@ public interface Environment {
/** /**
* Returns the IDs of all the output channels connected to the gate with the given ID. * Returns the IDs of all the output channels connected to the gate with the given ID.
* *
* @param gateID * @param gateID the gate ID
* the gate ID
* @return the IDs of all the output channels connected to the gate with the given ID * @return the IDs of all the output channels connected to the gate with the given ID
*/ */
Set<ChannelID> getOutputChannelIDsOfGate(GateID gateID); Set<ChannelID> getOutputChannelIDsOfGate(GateID gateID);
...@@ -202,8 +203,7 @@ public interface Environment { ...@@ -202,8 +203,7 @@ public interface Environment {
/** /**
* Returns the IDs of all the input channels connected to the gate with the given ID. * Returns the IDs of all the input channels connected to the gate with the given ID.
* *
* @param gateID * @param gateID the gate ID
* the gate ID
* @return the IDs of all the input channels connected to the gate with the given ID * @return the IDs of all the input channels connected to the gate with the given ID
*/ */
Set<ChannelID> getInputChannelIDsOfGate(GateID gateID); Set<ChannelID> getInputChannelIDsOfGate(GateID gateID);
...@@ -215,11 +215,13 @@ public interface Environment { ...@@ -215,11 +215,13 @@ public interface Environment {
/** /**
* Returns the buffer provider for this environment. * Returns the buffer provider for this environment.
* <p> * <p/>
* The returned buffer provider is used by the output side of the network stack. * The returned buffer provider is used by the output side of the network stack.
* *
* @return Buffer provider for the output side of the network stack * @return Buffer provider for the output side of the network stack
* @see eu.stratosphere.runtime.io.api.RecordWriter * @see eu.stratosphere.runtime.io.api.RecordWriter
*/ */
BufferProvider getOutputBufferProvider(); BufferProvider getOutputBufferProvider();
Map<String, FutureTask<Path>> getCopyTask();
} }
...@@ -14,6 +14,8 @@ ...@@ -14,6 +14,8 @@
package eu.stratosphere.nephele.execution; package eu.stratosphere.nephele.execution;
import static eu.stratosphere.nephele.execution.ExecutionState.FAILED; import static eu.stratosphere.nephele.execution.ExecutionState.FAILED;
import static eu.stratosphere.nephele.execution.ExecutionState.CANCELED;
import static eu.stratosphere.nephele.execution.ExecutionState.CANCELING;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
......
...@@ -13,30 +13,11 @@ ...@@ -13,30 +13,11 @@
package eu.stratosphere.nephele.execution; package eu.stratosphere.nephele.execution;
import java.io.IOException;
import java.util.ArrayDeque;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.FutureTask;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import eu.stratosphere.configuration.Configuration; import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.core.fs.Path; import eu.stratosphere.core.fs.Path;
import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.IOReadableWritable;
import eu.stratosphere.nephele.deployment.ChannelDeploymentDescriptor;
import eu.stratosphere.nephele.deployment.GateDeploymentDescriptor;
import eu.stratosphere.nephele.deployment.TaskDeploymentDescriptor; import eu.stratosphere.nephele.deployment.TaskDeploymentDescriptor;
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.nephele.protocols.AccumulatorProtocol; import eu.stratosphere.nephele.protocols.AccumulatorProtocol;
import eu.stratosphere.nephele.services.iomanager.IOManager; import eu.stratosphere.nephele.services.iomanager.IOManager;
...@@ -45,7 +26,6 @@ import eu.stratosphere.nephele.template.AbstractInvokable; ...@@ -45,7 +26,6 @@ import eu.stratosphere.nephele.template.AbstractInvokable;
import eu.stratosphere.nephele.template.InputSplitProvider; import eu.stratosphere.nephele.template.InputSplitProvider;
import eu.stratosphere.runtime.io.Buffer; import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.channels.ChannelID; import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.runtime.io.channels.OutputChannel; import eu.stratosphere.runtime.io.channels.OutputChannel;
import eu.stratosphere.runtime.io.gates.GateID; import eu.stratosphere.runtime.io.gates.GateID;
import eu.stratosphere.runtime.io.gates.InputGate; import eu.stratosphere.runtime.io.gates.InputGate;
...@@ -56,12 +36,27 @@ import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool; ...@@ -56,12 +36,27 @@ import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool;
import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPool; import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPool;
import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPoolOwner; import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPoolOwner;
import eu.stratosphere.util.StringUtils; import eu.stratosphere.util.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.io.IOException;
import java.util.ArrayDeque;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.FutureTask;
/** /**
* The user code of every Nephele task runs inside a <code>RuntimeEnvironment</code> object. The environment provides * The user code of every Nephele task runs inside a <code>RuntimeEnvironment</code> object. The environment provides
* important services to the task. It keeps track of setting up the communication channels and provides access to input * important services to the task. It keeps track of setting up the communication channels and provides access to input
* splits, memory manager, etc. * splits, memory manager, etc.
* <p> * <p/>
* This class is thread-safe. * This class is thread-safe.
*/ */
public class RuntimeEnvironment implements Environment, BufferProvider, LocalBufferPoolOwner, Runnable { public class RuntimeEnvironment implements Environment, BufferProvider, LocalBufferPoolOwner, Runnable {
...@@ -164,22 +159,18 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf ...@@ -164,22 +159,18 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
private LocalBufferPool outputBufferPool; private LocalBufferPool outputBufferPool;
private Map<String,FutureTask<Path>> cacheCopyTasks = new HashMap<String, FutureTask<Path>>();
/** /**
* Creates a new runtime environment object which contains the runtime information for the encapsulated Nephele * Creates a new runtime environment object which contains the runtime information for the encapsulated Nephele
* task. * task.
* *
* @param jobID * @param jobID the ID of the original Nephele job
* the ID of the original Nephele job * @param taskName the name of task running in this environment
* @param taskName * @param invokableClass invokableClass the class that should be instantiated as a Nephele task
* the name of task running in this environment * @param taskConfiguration the configuration object which was attached to the original JobVertex
* @param invokableClass * @param jobConfiguration the configuration object which was attached to the original JobGraph
* invokableClass the class that should be instantiated as a Nephele task * @throws Exception thrown if an error occurs while instantiating the invokable class
* @param taskConfiguration
* the configuration object which was attached to the original JobVertex
* @param jobConfiguration
* the configuration object which was attached to the original {@link JobGraph}
* @throws Exception
* thrown if an error occurs while instantiating the invokable class
*/ */
public RuntimeEnvironment(final JobID jobID, final String taskName, public RuntimeEnvironment(final JobID jobID, final String taskName,
final Class<? extends AbstractInvokable> invokableClass, final Configuration taskConfiguration, final Class<? extends AbstractInvokable> invokableClass, final Configuration taskConfiguration,
...@@ -204,18 +195,13 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf ...@@ -204,18 +195,13 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
/** /**
* Constructs a runtime environment from a task deployment description. * Constructs a runtime environment from a task deployment description.
* *
* @param tdd * @param tdd the task deployment description
* the task deployment description * @param memoryManager the task manager's memory manager component
* @param memoryManager * @param ioManager the task manager's I/O manager component
* the task manager's memory manager component * @param inputSplitProvider the input split provider for this environment
* @param ioManager * @throws Exception thrown if an error occurs while instantiating the invokable class
* the task manager's I/O manager component */
* @param inputSplitProvider @SuppressWarnings({"unchecked", "rawtypes"})
* the input split provider for this environment
* @throws Exception
* thrown if an error occurs while instantiating the invokable class
*/
@SuppressWarnings({ "unchecked", "rawtypes" })
public RuntimeEnvironment(final TaskDeploymentDescriptor tdd, public RuntimeEnvironment(final TaskDeploymentDescriptor tdd,
final MemoryManager memoryManager, final IOManager ioManager, final MemoryManager memoryManager, final IOManager ioManager,
final InputSplitProvider inputSplitProvider, final InputSplitProvider inputSplitProvider,
...@@ -246,7 +232,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf ...@@ -246,7 +232,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
int numInputGates = tdd.getNumberOfInputGateDescriptors(); int numInputGates = tdd.getNumberOfInputGateDescriptors();
for(int i = 0; i < numInputGates; i++){ for (int i = 0; i < numInputGates; i++) {
this.inputGates.get(i).initializeChannels(tdd.getInputGateDescriptor(i)); this.inputGates.get(i).initializeChannels(tdd.getInputGateDescriptor(i));
} }
} }
...@@ -318,7 +304,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf ...@@ -318,7 +304,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
if (this.executionObserver.isCanceled() || t instanceof CancelTaskException) { if (this.executionObserver.isCanceled() || t instanceof CancelTaskException) {
changeExecutionState(ExecutionState.CANCELED, null); changeExecutionState(ExecutionState.CANCELED, null);
} else { }
else {
changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(t)); changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(t));
} }
...@@ -347,7 +334,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf ...@@ -347,7 +334,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
if (this.executionObserver.isCanceled() || t instanceof CancelTaskException) { if (this.executionObserver.isCanceled() || t instanceof CancelTaskException) {
changeExecutionState(ExecutionState.CANCELED, null); changeExecutionState(ExecutionState.CANCELED, null);
} else { }
else {
changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(t)); changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(t));
} }
...@@ -401,8 +389,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf ...@@ -401,8 +389,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
/** /**
* Returns the registered input gate with index <code>pos</code>. * Returns the registered input gate with index <code>pos</code>.
* *
* @param pos * @param pos the index of the input gate to return
* the index of the input gate to return
* @return the input gate at index <code>pos</code> or <code>null</code> if no such index exists * @return the input gate at index <code>pos</code> or <code>null</code> if no such index exists
*/ */
public InputGate<? extends IOReadableWritable> getInputGate(final int pos) { public InputGate<? extends IOReadableWritable> getInputGate(final int pos) {
...@@ -416,8 +403,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf ...@@ -416,8 +403,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
/** /**
* Returns the registered output gate with index <code>pos</code>. * Returns the registered output gate with index <code>pos</code>.
* *
* @param index * @param index the index of the output gate to return
* the index of the output gate to return
* @return the output gate at index <code>pos</code> or <code>null</code> if no such index exists * @return the output gate at index <code>pos</code> or <code>null</code> if no such index exists
*/ */
public OutputGate getOutputGate(int index) { public OutputGate getOutputGate(int index) {
...@@ -439,7 +425,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf ...@@ -439,7 +425,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
if (this.executingThread == null) { if (this.executingThread == null) {
if (this.taskName == null) { if (this.taskName == null) {
this.executingThread = new Thread(this); this.executingThread = new Thread(this);
} else { }
else {
this.executingThread = new Thread(this, getTaskNameWithIndex()); this.executingThread = new Thread(this, getTaskNameWithIndex());
} }
} }
...@@ -451,10 +438,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf ...@@ -451,10 +438,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
/** /**
* Blocks until all output channels are closed. * Blocks until all output channels are closed.
* *
* @throws IOException * @throws IOException thrown if an error occurred while closing the output channels
* thrown if an error occurred while closing the output channels * @throws InterruptedException thrown if the thread waiting for the channels to be closed is interrupted
* @throws InterruptedException
* thrown if the thread waiting for the channels to be closed is interrupted
*/ */
private void waitForOutputChannelsToBeClosed() throws InterruptedException { private void waitForOutputChannelsToBeClosed() throws InterruptedException {
// Make sure, we leave this method with an InterruptedException when the task has been canceled // Make sure, we leave this method with an InterruptedException when the task has been canceled
...@@ -470,10 +455,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf ...@@ -470,10 +455,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
/** /**
* Blocks until all input channels are closed. * Blocks until all input channels are closed.
* *
* @throws IOException * @throws IOException thrown if an error occurred while closing the input channels
* thrown if an error occurred while closing the input channels * @throws InterruptedException thrown if the thread waiting for the channels to be closed is interrupted
* @throws InterruptedException
* thrown if the thread waiting for the channels to be closed is interrupted
*/ */
private void waitForInputChannelsToBeClosed() throws IOException, InterruptedException { private void waitForInputChannelsToBeClosed() throws IOException, InterruptedException {
// Wait for disconnection of all output gates // Wait for disconnection of all output gates
...@@ -494,7 +477,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf ...@@ -494,7 +477,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
if (allClosed) { if (allClosed) {
break; break;
} else { }
else {
Thread.sleep(SLEEPINTERVAL); Thread.sleep(SLEEPINTERVAL);
} }
} }
...@@ -574,8 +558,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf ...@@ -574,8 +558,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
/** /**
* Sets the execution observer for this environment. * Sets the execution observer for this environment.
* *
* @param executionObserver * @param executionObserver the execution observer for this environment
* the execution observer for this environment
*/ */
public void setExecutionObserver(final ExecutionObserver executionObserver) { public void setExecutionObserver(final ExecutionObserver executionObserver) {
this.executionObserver = executionObserver; this.executionObserver = executionObserver;
...@@ -616,7 +599,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf ...@@ -616,7 +599,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
@Override @Override
public Set<ChannelID> getOutputChannelIDs() { public Set<ChannelID> getOutputChannelIDs() {
Set<ChannelID> ids= new HashSet<ChannelID>(); Set<ChannelID> ids = new HashSet<ChannelID>();
for (OutputGate gate : this.outputGates) { for (OutputGate gate : this.outputGates) {
for (OutputChannel channel : gate.channels()) { for (OutputChannel channel : gate.channels()) {
...@@ -732,6 +715,15 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf ...@@ -732,6 +715,15 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
return accumulatorProtocolProxy; return accumulatorProtocolProxy;
} }
public void addCopyTaskForCacheFile(String name, FutureTask<Path> copyTask) {
this.cacheCopyTasks.put(name, copyTask);
}
@Override
public Map<String, FutureTask<Path>> getCopyTask() {
return this.cacheCopyTasks;
}
@Override @Override
public BufferProvider getOutputBufferProvider() { public BufferProvider getOutputBufferProvider() {
return this; return this;
...@@ -757,7 +749,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf ...@@ -757,7 +749,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
} }
@Override @Override
public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) { public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
return this.outputBufferPool.registerBufferAvailabilityListener(listener); return this.outputBufferPool.registerBufferAvailabilityListener(listener);
} }
......
...@@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicBoolean; ...@@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
import eu.stratosphere.nephele.taskmanager.TaskKillResult;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
...@@ -690,6 +691,41 @@ public final class ExecutionVertex { ...@@ -690,6 +691,41 @@ public final class ExecutionVertex {
} }
} }
/**
* Kills and removes the task represented by this vertex from the instance it is currently running on. If the
* corresponding task is not in the state <code>RUNNING</code>, this call will be ignored. If the call has been
* executed
* successfully, the task will change the state <code>FAILED</code>.
*
* @return the result of the task kill attempt
*/
public TaskKillResult killTask() {
final ExecutionState state = this.executionState.get();
if (state != ExecutionState.RUNNING) {
final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.ILLEGAL_STATE);
result.setDescription("Vertex " + this.toString() + " is in state " + state);
return result;
}
final AllocatedResource ar = this.allocatedResource.get();
if (ar == null) {
final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.NO_INSTANCE);
result.setDescription("Assigned instance of vertex " + this.toString() + " is null!");
return result;
}
try {
return ar.getInstance().killTask(this.vertexID);
} catch (IOException e) {
final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.IPC_ERROR);
result.setDescription(StringUtils.stringifyException(e));
return result;
}
}
/** /**
* Cancels and removes the task represented by this vertex * Cancels and removes the task represented by this vertex
* from the instance it is currently running on. If the task * from the instance it is currently running on. If the task
......
...@@ -24,6 +24,7 @@ import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest ...@@ -24,6 +24,7 @@ import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse;
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate;
import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
import eu.stratosphere.nephele.taskmanager.TaskKillResult;
import eu.stratosphere.runtime.io.channels.ChannelID; import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.nephele.ipc.RPC; import eu.stratosphere.nephele.ipc.RPC;
import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobgraph.JobID;
...@@ -205,6 +206,21 @@ public abstract class AbstractInstance extends NetworkNode { ...@@ -205,6 +206,21 @@ public abstract class AbstractInstance extends NetworkNode {
return getTaskManagerProxy().cancelTask(id); return getTaskManagerProxy().cancelTask(id);
} }
/**
* Kills the task identified by the given ID at the instance's
* {@link eu.stratosphere.nephele.taskmanager.TaskManager}.
*
* @param id
* the ID identifying the task to be killed
* @throws IOException
* thrown if an error occurs while transmitting the request or receiving the response
* @return the result of the kill attempt
*/
public synchronized TaskKillResult killTask(final ExecutionVertexID id) throws IOException {
return getTaskManagerProxy().killTask(id);
}
@Override @Override
public boolean equals(final Object obj) { public boolean equals(final Object obj) {
......
...@@ -13,11 +13,10 @@ ...@@ -13,11 +13,10 @@
package eu.stratosphere.nephele.jobgraph; package eu.stratosphere.nephele.jobgraph;
import java.nio.ByteBuffer; import eu.stratosphere.nephele.AbstractID;
import javax.xml.bind.DatatypeConverter; import javax.xml.bind.DatatypeConverter;
import java.nio.ByteBuffer;
import eu.stratosphere.nephele.AbstractID;
public final class JobID extends AbstractID { public final class JobID extends AbstractID {
...@@ -44,6 +43,12 @@ public final class JobID extends AbstractID { ...@@ -44,6 +43,12 @@ public final class JobID extends AbstractID {
return new JobID(bytes); return new JobID(bytes);
} }
public static JobID fromByteBuffer(ByteBuffer buf) {
long lower = buf.getLong();
long upper = buf.getLong();
return new JobID(lower, upper);
}
public static JobID fromByteBuffer(ByteBuffer buf, int offset) { public static JobID fromByteBuffer(ByteBuffer buf, int offset) {
long lower = buf.getLong(offset); long lower = buf.getLong(offset);
long upper = buf.getLong(offset + 8); long upper = buf.getLong(offset + 8);
......
...@@ -31,6 +31,8 @@ import java.util.concurrent.Executors; ...@@ -31,6 +31,8 @@ import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import eu.stratosphere.nephele.managementgraph.ManagementVertexID;
import eu.stratosphere.nephele.taskmanager.TaskKillResult;
import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.CommandLineParser; import org.apache.commons.cli.CommandLineParser;
import org.apache.commons.cli.GnuParser; import org.apache.commons.cli.GnuParser;
...@@ -892,6 +894,38 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol ...@@ -892,6 +894,38 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
return eventList; return eventList;
} }
@Override
public void killTask(final JobID jobID, final ManagementVertexID id) throws IOException {
final ExecutionGraph eg = this.scheduler.getExecutionGraphByID(jobID);
if (eg == null) {
LOG.error("Cannot find execution graph for job " + jobID);
return;
}
final ExecutionVertex vertex = eg.getVertexByID(ExecutionVertexID.fromManagementVertexID(id));
if (vertex == null) {
LOG.error("Cannot find execution vertex with ID " + id);
return;
}
LOG.info("Killing task " + vertex + " of job " + jobID);
final Runnable runnable = new Runnable() {
@Override
public void run() {
final TaskKillResult result = vertex.killTask();
if (result.getReturnCode() != AbstractTaskResult.ReturnCode.SUCCESS) {
LOG.error(result.getDescription());
}
}
};
eg.executeCommand(runnable);
}
@Override @Override
public void killInstance(final StringRecord instanceName) throws IOException { public void killInstance(final StringRecord instanceName) throws IOException {
......
...@@ -21,6 +21,7 @@ import java.util.Map; ...@@ -21,6 +21,7 @@ import java.util.Map;
import java.util.Queue; import java.util.Queue;
import java.util.Set; import java.util.Set;
import eu.stratosphere.nephele.taskmanager.AbstractTaskResult;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
...@@ -87,8 +88,8 @@ public final class RecoveryLogic { ...@@ -87,8 +88,8 @@ public final class RecoveryLogic {
verticesToBeRestarted.put(vertex.getID(), vertex); verticesToBeRestarted.put(vertex.getID(), vertex);
final TaskCancelResult cancelResult = vertex.cancelTask(); final TaskCancelResult cancelResult = vertex.cancelTask();
if (cancelResult.getReturnCode() != ReturnCode.SUCCESS if (cancelResult.getReturnCode() != AbstractTaskResult.ReturnCode.SUCCESS
&& cancelResult.getReturnCode() != ReturnCode.TASK_NOT_FOUND) { && cancelResult.getReturnCode() != AbstractTaskResult.ReturnCode.TASK_NOT_FOUND) {
verticesToBeRestarted.remove(vertex.getID()); verticesToBeRestarted.remove(vertex.getID());
LOG.error("Unable to cancel vertex" + cancelResult.getDescription()); LOG.error("Unable to cancel vertex" + cancelResult.getDescription());
......
...@@ -24,6 +24,7 @@ import eu.stratosphere.nephele.instance.InstanceType; ...@@ -24,6 +24,7 @@ import eu.stratosphere.nephele.instance.InstanceType;
import eu.stratosphere.nephele.instance.InstanceTypeDescription; import eu.stratosphere.nephele.instance.InstanceTypeDescription;
import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.nephele.managementgraph.ManagementGraph; import eu.stratosphere.nephele.managementgraph.ManagementGraph;
import eu.stratosphere.nephele.managementgraph.ManagementVertexID;
import eu.stratosphere.nephele.topology.NetworkTopology; import eu.stratosphere.nephele.topology.NetworkTopology;
/** /**
...@@ -80,6 +81,18 @@ public interface ExtendedManagementProtocol extends JobManagementProtocol { ...@@ -80,6 +81,18 @@ public interface ExtendedManagementProtocol extends JobManagementProtocol {
*/ */
List<AbstractEvent> getEvents(JobID jobID) throws IOException; List<AbstractEvent> getEvents(JobID jobID) throws IOException;
/**
* Kills the task with the given vertex ID.
*
* @param jobID
* the ID of the job the vertex to be killed belongs to
* @param id
* the vertex ID which identified the task be killed
* @throws IOException
* thrown if an error occurs while transmitting the kill request
*/
void killTask(JobID jobID, ManagementVertexID id) throws IOException;
/** /**
* Kills the instance with the given name (i.e. shuts down its task manager). * Kills the instance with the given name (i.e. shuts down its task manager).
* *
......
...@@ -23,6 +23,7 @@ import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest ...@@ -23,6 +23,7 @@ import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse;
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate;
import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
import eu.stratosphere.nephele.taskmanager.TaskKillResult;
import eu.stratosphere.runtime.io.channels.ChannelID; import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.nephele.taskmanager.TaskCancelResult; import eu.stratosphere.nephele.taskmanager.TaskCancelResult;
import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult; import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult;
...@@ -57,6 +58,17 @@ public interface TaskOperationProtocol extends VersionedProtocol { ...@@ -57,6 +58,17 @@ public interface TaskOperationProtocol extends VersionedProtocol {
*/ */
TaskCancelResult cancelTask(ExecutionVertexID id) throws IOException; TaskCancelResult cancelTask(ExecutionVertexID id) throws IOException;
/**
* Advises the task manager to kill the task with the given ID.
*
* @param id
* the ID of the task to kill
* @return the result of the task kill attempt
* @throws IOException
* thrown if an error occurs during this remote procedure call
*/
TaskKillResult killTask(ExecutionVertexID id) throws IOException;
/** /**
* Queries the task manager about the cache status of the libraries stated in the {@link LibraryCacheProfileRequest} * Queries the task manager about the cache status of the libraries stated in the {@link LibraryCacheProfileRequest}
* object. * object.
......
...@@ -25,6 +25,7 @@ import eu.stratosphere.nephele.jobgraph.JobID; ...@@ -25,6 +25,7 @@ import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.nephele.profiling.TaskManagerProfiler; import eu.stratosphere.nephele.profiling.TaskManagerProfiler;
import eu.stratosphere.nephele.services.memorymanager.MemoryManager; import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
import eu.stratosphere.nephele.template.AbstractInvokable; import eu.stratosphere.nephele.template.AbstractInvokable;
import eu.stratosphere.util.StringUtils;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
...@@ -57,9 +58,7 @@ public final class Task implements ExecutionObserver { ...@@ -57,9 +58,7 @@ public final class Task implements ExecutionObserver {
private Queue<ExecutionListener> registeredListeners = new ConcurrentLinkedQueue<ExecutionListener>(); private Queue<ExecutionListener> registeredListeners = new ConcurrentLinkedQueue<ExecutionListener>();
public Task(final ExecutionVertexID vertexID, final RuntimeEnvironment environment, public Task(ExecutionVertexID vertexID, final RuntimeEnvironment environment, TaskManager taskManager) {
final TaskManager taskManager) {
this.vertexID = vertexID; this.vertexID = vertexID;
this.environment = environment; this.environment = environment;
this.taskManager = taskManager; this.taskManager = taskManager;
...@@ -102,42 +101,33 @@ public final class Task implements ExecutionObserver { ...@@ -102,42 +101,33 @@ public final class Task implements ExecutionObserver {
executionStateChanged(ExecutionState.FAILED, "Execution thread died unexpectedly"); executionStateChanged(ExecutionState.FAILED, "Execution thread died unexpectedly");
} }
/** public void cancelExecution() {
* Checks if the state of the thread which is associated with this task is <code>TERMINATED</code>. cancelOrKillExecution(true);
*
* @return <code>true</code> if the state of this thread which is associated with this task is
* <code>TERMINATED</code>, <code>false</code> otherwise
*/
public boolean isTerminated() {
final Thread executingThread = this.environment.getExecutingThread();
if (executingThread.getState() == Thread.State.TERMINATED) {
return true;
}
return false;
} }
/** public void killExecution() {
* Starts the execution of this task. cancelOrKillExecution(false);
*/
public void startExecution() {
final Thread thread = this.environment.getExecutingThread();
thread.start();
} }
/** /**
* Cancels the execution of the task (i.e. interrupts the execution thread). * Cancels or kills the task.
*
* @param cancel <code>true/code> if the task shall be canceled, <code>false</code> if it shall be killed
*/ */
public void cancelExecution() { private void cancelOrKillExecution(boolean cancel) {
final Thread executingThread = this.environment.getExecutingThread(); final Thread executingThread = this.environment.getExecutingThread();
if (executingThread == null) { if (executingThread == null) {
return; return;
} }
LOG.info("Canceling " + this.environment.getTaskNameWithIndex()); if (this.executionState != ExecutionState.RUNNING && this.executionState != ExecutionState.FINISHING) {
return;
}
LOG.info((cancel ? "Canceling " : "Killing ") + this.environment.getTaskNameWithIndex());
if (cancel) {
this.isCanceled = true; this.isCanceled = true;
// Change state // Change state
executionStateChanged(ExecutionState.CANCELING, null); executionStateChanged(ExecutionState.CANCELING, null);
...@@ -149,11 +139,13 @@ public final class Task implements ExecutionObserver { ...@@ -149,11 +139,13 @@ public final class Task implements ExecutionObserver {
invokable.cancel(); invokable.cancel();
} }
} catch (Throwable e) { } catch (Throwable e) {
LOG.error("Error while canceling task", e); LOG.error(StringUtils.stringifyException(e));
}
} }
// Continuously interrupt the user thread until it changed to state CANCELED // Continuously interrupt the user thread until it changed to state CANCELED
while (true) { while (true) {
executingThread.interrupt(); executingThread.interrupt();
if (!executingThread.isAlive()) { if (!executingThread.isAlive()) {
...@@ -168,12 +160,36 @@ public final class Task implements ExecutionObserver { ...@@ -168,12 +160,36 @@ public final class Task implements ExecutionObserver {
break; break;
} }
if (LOG.isDebugEnabled()) if (LOG.isDebugEnabled()) {
LOG.debug("Sending repeated canceling signal to " + LOG.debug("Sending repeated " + (cancel == true ? "canceling" : "killing") + " signal to " +
this.environment.getTaskName() + " with state " + this.executionState); this.environment.getTaskName() + " with state " + this.executionState);
} }
} }
}
/**
* Checks if the state of the thread which is associated with this task is <code>TERMINATED</code>.
*
* @return <code>true</code> if the state of this thread which is associated with this task is
* <code>TERMINATED</code>, <code>false</code> otherwise
*/
public boolean isTerminated() {
final Thread executingThread = this.environment.getExecutingThread();
if (executingThread.getState() == Thread.State.TERMINATED) {
return true;
}
return false;
}
/**
* Starts the execution of this task.
*/
public void startExecution() {
final Thread thread = this.environment.getExecutingThread();
thread.start();
}
/** /**
* Registers the task manager profiler with the task. * Registers the task manager profiler with the task.
...@@ -324,5 +340,4 @@ public final class Task implements ExecutionObserver { ...@@ -324,5 +340,4 @@ public final class Task implements ExecutionObserver {
return this.environment; return this.environment;
} }
} }
...@@ -11,43 +11,34 @@ ...@@ -11,43 +11,34 @@
* specific language governing permissions and limitations under the License. * specific language governing permissions and limitations under the License.
**********************************************************************************************************************/ **********************************************************************************************************************/
package eu.stratosphere.runtime.io.network.envelope; package eu.stratosphere.nephele.taskmanager;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
/** /**
* This exception is thrown to indicate that the deserialization process of a {@link Envelope} could not be * A <code>TaskKillResult</code> is used to report the results
* continued because a {@link Buffer} to store the envelope's content is currently not available. * of a task kill attempt. It contains the ID of the task to be killed, a return code and
* a description. In case of an error during the kill operation the description includes an error message.
* *
*/ */
public final class NoBufferAvailableException extends Exception { public class TaskKillResult extends AbstractTaskResult {
/** /**
* Generated serial UID. * Constructs a new task kill result.
*/
private static final long serialVersionUID = -9164212953646457026L;
/**
* The buffer provider which could not deliver a buffer.
*/
private final BufferProvider bufferProvider;
/**
* Constructs a new exception.
* *
* @param bufferProvider * @param vertexID
* the buffer provider which could not deliver a buffer * the task ID this result belongs to
* @param returnCode
* the return code of the kill
*/ */
public NoBufferAvailableException(final BufferProvider bufferProvider) { public TaskKillResult(final ExecutionVertexID vertexID, final ReturnCode returnCode) {
this.bufferProvider = bufferProvider; super(vertexID, returnCode);
} }
/** /**
* Returns the buffer provider which could not deliver a buffer. * Constructs an empty task kill result.
*
* @return the buffer provider which could not deliver a buffer
*/ */
public BufferProvider getBufferProvider() { public TaskKillResult() {
return this.bufferProvider; super();
} }
} }
...@@ -126,7 +126,7 @@ public class TaskManager implements TaskOperationProtocol { ...@@ -126,7 +126,7 @@ public class TaskManager implements TaskOperationProtocol {
private final InstanceConnectionInfo localInstanceConnectionInfo; private final InstanceConnectionInfo localInstanceConnectionInfo;
/** /**
* The instance of the {@link eu.stratosphere.nephele.taskmanager.io.bytebuffered.ChannelManager} which is responsible for * The instance of the {@link ChannelManager} which is responsible for
* setting up and cleaning up the byte buffered channels of the tasks. * setting up and cleaning up the byte buffered channels of the tasks.
*/ */
private final ChannelManager channelManager; private final ChannelManager channelManager;
...@@ -286,15 +286,31 @@ public class TaskManager implements TaskOperationProtocol { ...@@ -286,15 +286,31 @@ public class TaskManager implements TaskOperationProtocol {
ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY, ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE); ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE);
// Initialize the byte buffered channel manager int numInThreads = GlobalConfiguration.getInteger(
ChannelManager channelManager = null; ConfigConstants.TASK_MANAGER_NETTY_NUM_IN_THREADS_KEY,
ConfigConstants.DEFAULT_TASK_MANAGER_NETTY_NUM_IN_THREADS);
int numOutThreads = GlobalConfiguration.getInteger(
ConfigConstants.TASK_MANAGER_NETTY_NUM_OUT_THREADS_KEY,
ConfigConstants.DEFAULT_TASK_MANAGER_NETTY_NUM_OUT_THREADS);
int lowWaterMark = GlobalConfiguration.getInteger(
ConfigConstants.TASK_MANAGER_NETTY_LOW_WATER_MARK,
ConfigConstants.DEFAULT_TASK_MANAGER_NETTY_LOW_WATER_MARK);
int highWaterMark = GlobalConfiguration.getInteger(
ConfigConstants.TASK_MANAGER_NETTY_HIGH_WATER_MARK,
ConfigConstants.DEFAULT_TASK_MANAGER_NETTY_HIGH_WATER_MARK);
// Initialize the channel manager
try { try {
channelManager = new ChannelManager(this.lookupService, this.localInstanceConnectionInfo, numBuffers, bufferSize); this.channelManager = new ChannelManager(
this.lookupService, this.localInstanceConnectionInfo,
numBuffers, bufferSize, numInThreads, numOutThreads, lowWaterMark, highWaterMark);
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.error(StringUtils.stringifyException(ioe)); LOG.error(StringUtils.stringifyException(ioe));
throw new Exception("Failed to instantiate Byte-buffered channel manager. " + ioe.getMessage(), ioe); throw new Exception("Failed to instantiate Byte-buffered channel manager. " + ioe.getMessage(), ioe);
} }
this.channelManager = channelManager;
{ {
HardwareDescription resources = HardwareDescriptionFactory.extractFromSystem(); HardwareDescription resources = HardwareDescriptionFactory.extractFromSystem();
...@@ -933,7 +949,7 @@ public class TaskManager implements TaskOperationProtocol { ...@@ -933,7 +949,7 @@ public class TaskManager implements TaskOperationProtocol {
} }
@Override @Override
public void logBufferUtilization() throws IOException { public void logBufferUtilization() {
this.channelManager.logBufferUtilization(); this.channelManager.logBufferUtilization();
} }
...@@ -956,7 +972,7 @@ public class TaskManager implements TaskOperationProtocol { ...@@ -956,7 +972,7 @@ public class TaskManager implements TaskOperationProtocol {
@Override @Override
public void invalidateLookupCacheEntries(final Set<ChannelID> channelIDs) throws IOException { public void invalidateLookupCacheEntries(final Set<ChannelID> channelIDs) throws IOException {
this.byteBufferedChannelManager.invalidateLookupCacheEntries(channelIDs); this.channelManager.invalidateLookupCacheEntries(channelIDs);
} }
/** /**
......
/***********************************************************************************************************************
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.nephele.taskmanager.runtime;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.atomic.AtomicInteger;
public class ExecutorThreadFactory implements ThreadFactory {
public static final ExecutorThreadFactory INSTANCE = new ExecutorThreadFactory();
private static final String THREAD_NAME = "Nephele Executor Thread ";
private final AtomicInteger threadNumber = new AtomicInteger(1);
private ExecutorThreadFactory() {}
public Thread newThread(Runnable target) {
Thread t = new Thread(target, THREAD_NAME + threadNumber.getAndIncrement());
t.setDaemon(true);
return t;
}
}
...@@ -21,7 +21,6 @@ import eu.stratosphere.api.common.typeutils.TypeSerializer; ...@@ -21,7 +21,6 @@ import eu.stratosphere.api.common.typeutils.TypeSerializer;
import eu.stratosphere.api.common.typeutils.TypeSerializerFactory; import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.core.memory.DataOutputView;
import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.Environment;
import eu.stratosphere.nephele.io.MutableReader;
import eu.stratosphere.pact.runtime.hash.CompactingHashTable; import eu.stratosphere.pact.runtime.hash.CompactingHashTable;
import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannel; import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannel;
import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannelBroker; import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannelBroker;
...@@ -36,6 +35,7 @@ import eu.stratosphere.pact.runtime.task.RegularPactTask; ...@@ -36,6 +35,7 @@ import eu.stratosphere.pact.runtime.task.RegularPactTask;
import eu.stratosphere.pact.runtime.task.ResettablePactDriver; import eu.stratosphere.pact.runtime.task.ResettablePactDriver;
import eu.stratosphere.pact.runtime.task.util.TaskConfig; import eu.stratosphere.pact.runtime.task.util.TaskConfig;
import eu.stratosphere.pact.runtime.udf.RuntimeUDFContext; import eu.stratosphere.pact.runtime.udf.RuntimeUDFContext;
import eu.stratosphere.runtime.io.api.MutableReader;
import eu.stratosphere.types.Value; import eu.stratosphere.types.Value;
import eu.stratosphere.util.Collector; import eu.stratosphere.util.Collector;
import eu.stratosphere.util.InstantiationUtil; import eu.stratosphere.util.InstantiationUtil;
...@@ -322,14 +322,15 @@ public abstract class AbstractIterativePactTask<S extends Function, OT> extends ...@@ -322,14 +322,15 @@ public abstract class AbstractIterativePactTask<S extends Function, OT> extends
* <p/> * <p/>
* This collector is used by {@link IterationIntermediatePactTask} or {@link IterationTailPactTask} to update the * This collector is used by {@link IterationIntermediatePactTask} or {@link IterationTailPactTask} to update the
* solution set of workset iterations. Depending on the task configuration, either a fast (non-probing) * solution set of workset iterations. Depending on the task configuration, either a fast (non-probing)
* {@link SolutionSetFastUpdateOutputCollector} or normal (re-probing) {@link SolutionSetUpdateOutputCollector} * {@link eu.stratosphere.pact.runtime.iterative.io.SolutionSetFastUpdateOutputCollector} or normal (re-probing)
* is created. * {@link SolutionSetUpdateOutputCollector} is created.
* <p/> * <p/>
* If a non-null delegate is given, the new {@link Collector} will write back to the solution set and also call * If a non-null delegate is given, the new {@link Collector} will write back to the solution set and also call
* collect(T) of the delegate. * collect(T) of the delegate.
* *
* @param delegate null -OR- a delegate collector to be called by the newly created collector * @param delegate null -OR- a delegate collector to be called by the newly created collector
* @return a new {@link SolutionSetFastUpdateOutputCollector} or {@link SolutionSetUpdateOutputCollector} * @return a new {@link eu.stratosphere.pact.runtime.iterative.io.SolutionSetFastUpdateOutputCollector} or
* {@link SolutionSetUpdateOutputCollector}
*/ */
protected Collector<OT> createSolutionSetUpdateOutputCollector(Collector<OT> delegate) { protected Collector<OT> createSolutionSetUpdateOutputCollector(Collector<OT> delegate) {
Broker<CompactingHashTable<?>> solutionSetBroker = SolutionSetBroker.instance(); Broker<CompactingHashTable<?>> solutionSetBroker = SolutionSetBroker.instance();
......
...@@ -17,6 +17,7 @@ import java.io.IOException; ...@@ -17,6 +17,7 @@ import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import eu.stratosphere.runtime.io.api.BufferWriter;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
...@@ -27,8 +28,6 @@ import eu.stratosphere.api.common.typeutils.TypeSerializer; ...@@ -27,8 +28,6 @@ import eu.stratosphere.api.common.typeutils.TypeSerializer;
import eu.stratosphere.api.common.typeutils.TypeSerializerFactory; import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
import eu.stratosphere.core.memory.DataInputView; import eu.stratosphere.core.memory.DataInputView;
import eu.stratosphere.core.memory.MemorySegment; import eu.stratosphere.core.memory.MemorySegment;
import eu.stratosphere.nephele.io.AbstractRecordWriter;
import eu.stratosphere.nephele.io.RecordWriter;
import eu.stratosphere.pact.runtime.hash.CompactingHashTable; import eu.stratosphere.pact.runtime.hash.CompactingHashTable;
import eu.stratosphere.pact.runtime.io.InputViewIterator; import eu.stratosphere.pact.runtime.io.InputViewIterator;
import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannel; import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannel;
......
...@@ -18,7 +18,6 @@ import eu.stratosphere.api.common.typeutils.TypeComparator; ...@@ -18,7 +18,6 @@ import eu.stratosphere.api.common.typeutils.TypeComparator;
import eu.stratosphere.runtime.io.api.ChannelSelector; import eu.stratosphere.runtime.io.api.ChannelSelector;
import eu.stratosphere.pact.runtime.plugable.SerializationDelegate; import eu.stratosphere.pact.runtime.plugable.SerializationDelegate;
public class OutputEmitter<T> implements ChannelSelector<SerializationDelegate<T>> { public class OutputEmitter<T> implements ChannelSelector<SerializationDelegate<T>> {
private final ShipStrategyType strategy; // the shipping strategy used by this output emitter private final ShipStrategyType strategy; // the shipping strategy used by this output emitter
......
...@@ -15,13 +15,10 @@ package eu.stratosphere.pact.runtime.shipping; ...@@ -15,13 +15,10 @@ package eu.stratosphere.pact.runtime.shipping;
import eu.stratosphere.api.common.distributions.DataDistribution; import eu.stratosphere.api.common.distributions.DataDistribution;
import eu.stratosphere.api.common.typeutils.TypeComparator; import eu.stratosphere.api.common.typeutils.TypeComparator;
import eu.stratosphere.nephele.io.ChannelSelector;
import eu.stratosphere.runtime.io.api.ChannelSelector; import eu.stratosphere.runtime.io.api.ChannelSelector;
import eu.stratosphere.pact.runtime.plugable.pactrecord.RecordComparator;
import eu.stratosphere.types.Key; import eu.stratosphere.types.Key;
import eu.stratosphere.types.Record; import eu.stratosphere.types.Record;
public class RecordOutputEmitter implements ChannelSelector<Record> { public class RecordOutputEmitter implements ChannelSelector<Record> {
// ------------------------------------------------------------------------ // ------------------------------------------------------------------------
......
...@@ -193,8 +193,9 @@ public class DataSinkTask<IT> extends AbstractOutputTask { ...@@ -193,8 +193,9 @@ public class DataSinkTask<IT> extends AbstractOutputTask {
} }
// drop, if the task was canceled // drop, if the task was canceled
else if (!this.taskCanceled) { else if (!this.taskCanceled) {
if (LOG.isErrorEnabled()) if (LOG.isErrorEnabled()) {
LOG.error(getLogString("Error in user code: " + ex.getMessage()), ex); LOG.error(getLogString("Error in user code: " + ex.getMessage()), ex);
}
throw ex; throw ex;
} }
} }
......
...@@ -22,9 +22,7 @@ import eu.stratosphere.api.common.typeutils.TypeComparator; ...@@ -22,9 +22,7 @@ import eu.stratosphere.api.common.typeutils.TypeComparator;
import eu.stratosphere.api.common.typeutils.TypeComparatorFactory; import eu.stratosphere.api.common.typeutils.TypeComparatorFactory;
import eu.stratosphere.api.common.typeutils.TypeSerializer; import eu.stratosphere.api.common.typeutils.TypeSerializer;
import eu.stratosphere.api.common.typeutils.TypeSerializerFactory; import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
import eu.stratosphere.configuration.ConfigConstants;
import eu.stratosphere.configuration.Configuration; import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.configuration.GlobalConfiguration;
import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.IOReadableWritable;
import eu.stratosphere.nephele.execution.CancelTaskException; import eu.stratosphere.nephele.execution.CancelTaskException;
import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.Environment;
...@@ -280,8 +278,9 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem ...@@ -280,8 +278,9 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
@Override @Override
public void invoke() throws Exception { public void invoke() throws Exception {
if (LOG.isDebugEnabled()) if (LOG.isDebugEnabled()) {
LOG.debug(formatLogString("Start task code.")); LOG.debug(formatLogString("Start task code."));
}
// whatever happens in this scope, make sure that the local strategies are cleaned up! // whatever happens in this scope, make sure that the local strategies are cleaned up!
// note that the initialization of the local strategies is in the try-finally block as well, // note that the initialization of the local strategies is in the try-finally block as well,
...@@ -390,8 +389,9 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem ...@@ -390,8 +389,9 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
public void cancel() throws Exception { public void cancel() throws Exception {
this.running = false; this.running = false;
if (LOG.isDebugEnabled()) if (LOG.isDebugEnabled()) {
LOG.debug(formatLogString("Cancelling task code")); LOG.debug(formatLogString("Cancelling task code"));
}
try { try {
if (this.driver != null) { if (this.driver != null) {
......
...@@ -40,8 +40,6 @@ public class Buffer { ...@@ -40,8 +40,6 @@ public class Buffer {
} }
/** /**
* NOTE: Requires that the reference counter was increased prior to the constructor call!
*
* @param toDuplicate Buffer instance to duplicate * @param toDuplicate Buffer instance to duplicate
*/ */
private Buffer(Buffer toDuplicate) { private Buffer(Buffer toDuplicate) {
...@@ -74,7 +72,8 @@ public class Buffer { ...@@ -74,7 +72,8 @@ public class Buffer {
} }
public void recycleBuffer() { public void recycleBuffer() {
if (this.referenceCounter.decrementAndGet() == 0) { int refCount = this.referenceCounter.decrementAndGet();
if (refCount == 0) {
this.recycler.recycle(this.memorySegment); this.recycler.recycle(this.memorySegment);
} }
} }
......
...@@ -17,8 +17,8 @@ import java.io.IOException; ...@@ -17,8 +17,8 @@ import java.io.IOException;
import eu.stratosphere.nephele.event.task.AbstractEvent; import eu.stratosphere.nephele.event.task.AbstractEvent;
import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.runtime.io.network.envelope.Envelope; import eu.stratosphere.runtime.io.network.Envelope;
import eu.stratosphere.runtime.io.network.envelope.EnvelopeDispatcher; import eu.stratosphere.runtime.io.network.EnvelopeDispatcher;
/** /**
* The base class for channel objects. * The base class for channel objects.
......
...@@ -13,10 +13,10 @@ ...@@ -13,10 +13,10 @@
package eu.stratosphere.runtime.io.channels; package eu.stratosphere.runtime.io.channels;
import java.nio.ByteBuffer;
import eu.stratosphere.nephele.AbstractID; import eu.stratosphere.nephele.AbstractID;
import java.nio.ByteBuffer;
public class ChannelID extends AbstractID { public class ChannelID extends AbstractID {
public ChannelID() { public ChannelID() {
...@@ -31,6 +31,12 @@ public class ChannelID extends AbstractID { ...@@ -31,6 +31,12 @@ public class ChannelID extends AbstractID {
super(bytes); super(bytes);
} }
public static ChannelID fromByteBuffer(ByteBuffer buf) {
long lower = buf.getLong();
long upper = buf.getLong();
return new ChannelID(lower, upper);
}
public static ChannelID fromByteBuffer(ByteBuffer buf, int offset) { public static ChannelID fromByteBuffer(ByteBuffer buf, int offset) {
long lower = buf.getLong(offset); long lower = buf.getLong(offset);
long upper = buf.getLong(offset + 8); long upper = buf.getLong(offset + 8);
......
...@@ -24,3 +24,8 @@ public enum ChannelType { ...@@ -24,3 +24,8 @@ public enum ChannelType {
/** In-memory channels */ /** In-memory channels */
IN_MEMORY IN_MEMORY
} }
...@@ -21,7 +21,7 @@ import eu.stratosphere.runtime.io.Buffer; ...@@ -21,7 +21,7 @@ import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.gates.InputChannelResult; import eu.stratosphere.runtime.io.gates.InputChannelResult;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener; import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider; import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
import eu.stratosphere.runtime.io.network.envelope.Envelope; import eu.stratosphere.runtime.io.network.Envelope;
import eu.stratosphere.runtime.io.gates.InputGate; import eu.stratosphere.runtime.io.gates.InputGate;
import eu.stratosphere.runtime.io.serialization.AdaptiveSpanningRecordDeserializer; import eu.stratosphere.runtime.io.serialization.AdaptiveSpanningRecordDeserializer;
import eu.stratosphere.runtime.io.serialization.RecordDeserializer; import eu.stratosphere.runtime.io.serialization.RecordDeserializer;
...@@ -30,7 +30,6 @@ import eu.stratosphere.runtime.io.serialization.RecordDeserializer.Deserializati ...@@ -30,7 +30,6 @@ import eu.stratosphere.runtime.io.serialization.RecordDeserializer.Deserializati
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import java.io.EOFException;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayDeque; import java.util.ArrayDeque;
import java.util.Arrays; import java.util.Arrays;
...@@ -83,6 +82,8 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen ...@@ -83,6 +82,8 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
private int lastReceivedEnvelope = -1; private int lastReceivedEnvelope = -1;
private ChannelID lastSourceID = null;
private boolean destroyCalled = false; private boolean destroyCalled = false;
// ---------------------- // ----------------------
...@@ -158,6 +159,9 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen ...@@ -158,6 +159,9 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
// public abstract AbstractTaskEvent getCurrentEvent(); // public abstract AbstractTaskEvent getCurrentEvent();
private DeserializationResult lastDeserializationResult;
public InputChannelResult readRecord(T target) throws IOException { public InputChannelResult readRecord(T target) throws IOException {
if (this.dataBuffer == null) { if (this.dataBuffer == null) {
if (isClosed()) { if (isClosed()) {
...@@ -176,7 +180,7 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen ...@@ -176,7 +180,7 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
{ {
// sanity check: an event may only come after a complete record. // sanity check: an event may only come after a complete record.
if (this.deserializer.hasUnfinishedData()) { if (this.deserializer.hasUnfinishedData()) {
throw new IOException("Channel received an event before completing the current partial record."); throw new IllegalStateException("Channel received an event before completing the current partial record.");
} }
AbstractEvent evt = boe.getEvent(); AbstractEvent evt = boe.getEvent();
...@@ -202,8 +206,8 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen ...@@ -202,8 +206,8 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
} }
} }
DeserializationResult deserializationResult = this.deserializer.getNextRecord(target); DeserializationResult deserializationResult = this.deserializer.getNextRecord(target);
this.lastDeserializationResult = deserializationResult;
if (deserializationResult.isBufferConsumed()) { if (deserializationResult.isBufferConsumed()) {
releasedConsumedReadBuffer(this.dataBuffer); releasedConsumedReadBuffer(this.dataBuffer);
...@@ -348,6 +352,7 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen ...@@ -348,6 +352,7 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
this.queuedEnvelopes.add(envelope); this.queuedEnvelopes.add(envelope);
this.lastReceivedEnvelope = sequenceNumber; this.lastReceivedEnvelope = sequenceNumber;
this.lastSourceID = envelope.getSource();
// Notify the channel about the new data. notify as much as there is (buffer plus once per event) // Notify the channel about the new data. notify as much as there is (buffer plus once per event)
if (envelope.getBuffer() != null) { if (envelope.getBuffer() != null) {
...@@ -432,7 +437,7 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen ...@@ -432,7 +437,7 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
} }
@Override @Override
public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) { public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
return this.inputGate.registerBufferAvailabilityListener(listener); return this.inputGate.registerBufferAvailabilityListener(listener);
} }
......
...@@ -17,7 +17,7 @@ import eu.stratosphere.nephele.event.task.AbstractEvent; ...@@ -17,7 +17,7 @@ import eu.stratosphere.nephele.event.task.AbstractEvent;
import eu.stratosphere.nephele.event.task.AbstractTaskEvent; import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.runtime.io.Buffer; import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.network.envelope.Envelope; import eu.stratosphere.runtime.io.network.Envelope;
import eu.stratosphere.runtime.io.gates.OutputGate; import eu.stratosphere.runtime.io.gates.OutputGate;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
...@@ -107,7 +107,6 @@ public class OutputChannel extends Channel { ...@@ -107,7 +107,6 @@ public class OutputChannel extends Channel {
this.receiverCloseRequested = true; this.receiverCloseRequested = true;
this.closeLock.notifyAll(); this.closeLock.notifyAll();
} }
LOG.debug("OutputChannel received close event from target.");
} }
else if (event instanceof AbstractTaskEvent) { else if (event instanceof AbstractTaskEvent) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
...@@ -165,7 +164,7 @@ public class OutputChannel extends Channel { ...@@ -165,7 +164,7 @@ public class OutputChannel extends Channel {
private void checkStatus() throws IOException { private void checkStatus() throws IOException {
if (this.senderCloseRequested) { if (this.senderCloseRequested) {
throw new IllegalStateException(String.format("Channel %s already requested to be closed.", getID())); throw new IllegalStateException(String.format("Channel %s already requested to be closed", getID()));
} }
if (this.receiverCloseRequested) { if (this.receiverCloseRequested) {
throw new ReceiverAlreadyClosedException(); throw new ReceiverAlreadyClosedException();
......
...@@ -14,9 +14,6 @@ ...@@ -14,9 +14,6 @@
package eu.stratosphere.runtime.io.gates; package eu.stratosphere.runtime.io.gates;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.BlockingQueue; import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
...@@ -24,7 +21,6 @@ import java.util.concurrent.atomic.AtomicReference; ...@@ -24,7 +21,6 @@ import java.util.concurrent.atomic.AtomicReference;
import eu.stratosphere.nephele.deployment.ChannelDeploymentDescriptor; import eu.stratosphere.nephele.deployment.ChannelDeploymentDescriptor;
import eu.stratosphere.nephele.deployment.GateDeploymentDescriptor; import eu.stratosphere.nephele.deployment.GateDeploymentDescriptor;
import eu.stratosphere.runtime.io.Buffer; import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener; import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider; import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool; import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool;
...@@ -38,7 +34,6 @@ import eu.stratosphere.nephele.event.task.AbstractEvent; ...@@ -38,7 +34,6 @@ import eu.stratosphere.nephele.event.task.AbstractEvent;
import eu.stratosphere.nephele.event.task.AbstractTaskEvent; import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
import eu.stratosphere.nephele.execution.Environment; import eu.stratosphere.nephele.execution.Environment;
import eu.stratosphere.runtime.io.channels.InputChannel; import eu.stratosphere.runtime.io.channels.InputChannel;
import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobgraph.JobID;
/** /**
...@@ -378,7 +373,7 @@ public class InputGate<T extends IOReadableWritable> extends Gate<T> implements ...@@ -378,7 +373,7 @@ public class InputGate<T extends IOReadableWritable> extends Gate<T> implements
} }
@Override @Override
public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) { public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
return this.bufferPool.registerBufferAvailabilityListener(listener); return this.bufferPool.registerBufferAvailabilityListener(listener);
} }
} }
...@@ -19,7 +19,6 @@ import eu.stratosphere.nephele.deployment.GateDeploymentDescriptor; ...@@ -19,7 +19,6 @@ import eu.stratosphere.nephele.deployment.GateDeploymentDescriptor;
import eu.stratosphere.nephele.event.task.AbstractEvent; import eu.stratosphere.nephele.event.task.AbstractEvent;
import eu.stratosphere.runtime.io.Buffer; import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.channels.ChannelID; import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.runtime.io.channels.OutputChannel; import eu.stratosphere.runtime.io.channels.OutputChannel;
import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobgraph.JobID;
......
...@@ -31,15 +31,13 @@ import eu.stratosphere.nephele.AbstractID; ...@@ -31,15 +31,13 @@ import eu.stratosphere.nephele.AbstractID;
import eu.stratosphere.runtime.io.Buffer; import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider; import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProviderBroker; import eu.stratosphere.runtime.io.network.bufferprovider.BufferProviderBroker;
import eu.stratosphere.runtime.io.network.bufferprovider.DiscardBufferPool;
import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool; import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool;
import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPoolOwner; import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPoolOwner;
import eu.stratosphere.runtime.io.network.bufferprovider.SerialSingleBufferPool;
import eu.stratosphere.runtime.io.network.envelope.Envelope;
import eu.stratosphere.runtime.io.network.envelope.EnvelopeDispatcher;
import eu.stratosphere.runtime.io.network.envelope.EnvelopeReceiverList;
import eu.stratosphere.runtime.io.gates.GateID; import eu.stratosphere.runtime.io.gates.GateID;
import eu.stratosphere.runtime.io.gates.InputGate; import eu.stratosphere.runtime.io.gates.InputGate;
import eu.stratosphere.runtime.io.gates.OutputGate; import eu.stratosphere.runtime.io.gates.OutputGate;
import eu.stratosphere.runtime.io.network.netty.NettyConnectionManager;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
...@@ -52,7 +50,7 @@ import java.util.concurrent.ConcurrentHashMap; ...@@ -52,7 +50,7 @@ import java.util.concurrent.ConcurrentHashMap;
/** /**
* The channel manager sets up the network buffers and dispatches data between channels. * The channel manager sets up the network buffers and dispatches data between channels.
*/ */
public final class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker { public class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker {
private static final Log LOG = LogFactory.getLog(ChannelManager.class); private static final Log LOG = LogFactory.getLog(ChannelManager.class);
...@@ -68,20 +66,27 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB ...@@ -68,20 +66,27 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
private final GlobalBufferPool globalBufferPool; private final GlobalBufferPool globalBufferPool;
private final NetworkConnectionManager networkConnectionManager; private final NettyConnectionManager nettyConnectionManager;
private final InetSocketAddress ourAddress; private final InetSocketAddress ourAddress;
private final SerialSingleBufferPool discardingDataPool; private final DiscardBufferPool discardBufferPool;
// ----------------------------------------------------------------------------------------------------------------- // -----------------------------------------------------------------------------------------------------------------
public ChannelManager(ChannelLookupProtocol channelLookupService, InstanceConnectionInfo connectionInfo, public ChannelManager(ChannelLookupProtocol channelLookupService, InstanceConnectionInfo connectionInfo,
int numNetworkBuffers, int networkBufferSize) throws IOException { int numNetworkBuffers, int networkBufferSize,
int numInThreads, int numOutThreads,
int lowWatermark, int highWaterMark) throws IOException {
this.channelLookupService = channelLookupService; this.channelLookupService = channelLookupService;
this.connectionInfo = connectionInfo; this.connectionInfo = connectionInfo;
this.globalBufferPool = new GlobalBufferPool(numNetworkBuffers, networkBufferSize); this.globalBufferPool = new GlobalBufferPool(numNetworkBuffers, networkBufferSize);
this.networkConnectionManager = new NetworkConnectionManager(this, connectionInfo.address(), connectionInfo.dataPort());
this.nettyConnectionManager = new NettyConnectionManager(
this, connectionInfo.address(), connectionInfo.dataPort(),
networkBufferSize, numInThreads, numOutThreads, lowWatermark, highWaterMark);
// management data structures // management data structures
this.channels = new ConcurrentHashMap<ChannelID, Channel>(); this.channels = new ConcurrentHashMap<ChannelID, Channel>();
...@@ -91,11 +96,11 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB ...@@ -91,11 +96,11 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
this.ourAddress = new InetSocketAddress(connectionInfo.address(), connectionInfo.dataPort()); this.ourAddress = new InetSocketAddress(connectionInfo.address(), connectionInfo.dataPort());
// a special pool if the data is to be discarded // a special pool if the data is to be discarded
this.discardingDataPool = new SerialSingleBufferPool(networkBufferSize); this.discardBufferPool = new DiscardBufferPool();
} }
public void shutdown() { public void shutdown() {
this.networkConnectionManager.shutDown(); this.nettyConnectionManager.shutdown();
this.globalBufferPool.destroy(); this.globalBufferPool.destroy();
} }
...@@ -301,7 +306,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB ...@@ -301,7 +306,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
} }
} }
private void generateSenderHint(Envelope envelope, RemoteReceiver receiver) { private void generateSenderHint(Envelope envelope, RemoteReceiver receiver) throws IOException {
Channel channel = this.channels.get(envelope.getSource()); Channel channel = this.channels.get(envelope.getSource());
if (channel == null) { if (channel == null) {
LOG.error("Cannot find channel for channel ID " + envelope.getSource()); LOG.error("Cannot find channel for channel ID " + envelope.getSource());
...@@ -319,7 +324,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB ...@@ -319,7 +324,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
final RemoteReceiver ourAddress = new RemoteReceiver(this.ourAddress, connectionIndex); final RemoteReceiver ourAddress = new RemoteReceiver(this.ourAddress, connectionIndex);
final Envelope senderHint = SenderHintEvent.createEnvelopeWithEvent(envelope, targetChannelID, ourAddress); final Envelope senderHint = SenderHintEvent.createEnvelopeWithEvent(envelope, targetChannelID, ourAddress);
this.networkConnectionManager.queueEnvelopeForTransfer(receiver, senderHint); this.nettyConnectionManager.enqueue(senderHint, receiver);
} }
/** /**
...@@ -331,7 +336,6 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB ...@@ -331,7 +336,6 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
* the source channel ID for which the receiver list shall be retrieved * the source channel ID for which the receiver list shall be retrieved
* @return the list of receivers or <code>null</code> if the receiver could not be determined * @return the list of receivers or <code>null</code> if the receiver could not be determined
* @throws IOException * @throws IOException
* @throws InterruptedException
*/ */
private EnvelopeReceiverList getReceiverList(JobID jobID, ChannelID sourceChannelID, boolean reportException) throws IOException { private EnvelopeReceiverList getReceiverList(JobID jobID, ChannelID sourceChannelID, boolean reportException) throws IOException {
EnvelopeReceiverList receiverList = this.receiverCache.get(sourceChannelID); EnvelopeReceiverList receiverList = this.receiverCache.get(sourceChannelID);
...@@ -383,8 +387,10 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB ...@@ -383,8 +387,10 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
this.receiverCache.put(sourceChannelID, receiverList); this.receiverCache.put(sourceChannelID, receiverList);
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Receivers for source channel ID " + sourceChannelID + " at task manager " + this.connectionInfo + LOG.debug(String.format("Receiver for %s: %s [%s])",
": " + receiverList); sourceChannelID,
receiverList.hasLocalReceiver() ? receiverList.getLocalReceiver() : receiverList.getRemoteReceiver(),
receiverList.hasLocalReceiver() ? "local" : "remote"));
} }
return receiverList; return receiverList;
...@@ -453,7 +459,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB ...@@ -453,7 +459,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
generateSenderHint(envelope, remoteReceiver); generateSenderHint(envelope, remoteReceiver);
} }
this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, envelope); this.nettyConnectionManager.enqueue(envelope, remoteReceiver);
success = true; success = true;
} }
} finally { } finally {
...@@ -501,7 +507,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB ...@@ -501,7 +507,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
generateSenderHint(envelope, remoteReceiver); generateSenderHint(envelope, remoteReceiver);
} }
this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, envelope); this.nettyConnectionManager.enqueue(envelope, remoteReceiver);
} }
} }
...@@ -597,7 +603,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB ...@@ -597,7 +603,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
// check if the receiver is already gone // check if the receiver is already gone
if (receiverList == null) { if (receiverList == null) {
return this.discardingDataPool; return this.discardBufferPool;
} }
if (!receiverList.hasLocalReceiver() || receiverList.hasRemoteReceiver()) { if (!receiverList.hasLocalReceiver() || receiverList.hasRemoteReceiver()) {
...@@ -610,7 +616,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB ...@@ -610,7 +616,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
if (channel == null) { if (channel == null) {
// receiver is already canceled // receiver is already canceled
return this.discardingDataPool; return this.discardBufferPool;
} }
if (!channel.isInputChannel()) { if (!channel.isInputChannel()) {
...@@ -633,8 +639,6 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB ...@@ -633,8 +639,6 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
bufferPool.logBufferUtilization(); bufferPool.logBufferUtilization();
} }
this.networkConnectionManager.logBufferUtilization();
System.out.println("\tIncoming connections:"); System.out.println("\tIncoming connections:");
for (Channel channel : this.channels.values()) { for (Channel channel : this.channels.values()) {
......
...@@ -11,7 +11,7 @@ ...@@ -11,7 +11,7 @@
* specific language governing permissions and limitations under the License. * specific language governing permissions and limitations under the License.
**********************************************************************************************************************/ **********************************************************************************************************************/
package eu.stratosphere.runtime.io.network.envelope; package eu.stratosphere.runtime.io.network;
import eu.stratosphere.nephele.event.task.AbstractEvent; import eu.stratosphere.nephele.event.task.AbstractEvent;
import eu.stratosphere.runtime.io.Buffer; import eu.stratosphere.runtime.io.Buffer;
...@@ -79,15 +79,17 @@ public final class Envelope { ...@@ -79,15 +79,17 @@ public final class Envelope {
} }
public void setEventsSerialized(ByteBuffer serializedEventList) { public void setEventsSerialized(ByteBuffer serializedEventList) {
if (this.serializedEventList != null) if (this.serializedEventList != null) {
throw new IllegalStateException("Event list has already been set."); throw new IllegalStateException("Event list has already been set.");
}
this.serializedEventList = serializedEventList; this.serializedEventList = serializedEventList;
} }
public void serializeEventList(List<? extends AbstractEvent> eventList) { public void serializeEventList(List<? extends AbstractEvent> eventList) {
if (this.serializedEventList != null) if (this.serializedEventList != null) {
throw new IllegalStateException("Event list has already been set."); throw new IllegalStateException("Event list has already been set.");
}
this.serializedEventList = serializeEvents(eventList); this.serializedEventList = serializeEvents(eventList);
} }
...@@ -166,4 +168,11 @@ public final class Envelope { ...@@ -166,4 +168,11 @@ public final class Envelope {
public boolean hasBuffer() { public boolean hasBuffer() {
return this.buffer != null; return this.buffer != null;
} }
@Override
public String toString() {
return String.format("Envelope %d [source id: %s, buffer size: %d, events size: %d]",
this.sequenceNumber, this.getSource(), this.buffer == null ? -1 : this.buffer.size(),
this.serializedEventList == null ? -1 : this.serializedEventList.remaining());
}
} }
...@@ -11,7 +11,7 @@ ...@@ -11,7 +11,7 @@
* specific language governing permissions and limitations under the License. * specific language governing permissions and limitations under the License.
**********************************************************************************************************************/ **********************************************************************************************************************/
package eu.stratosphere.runtime.io.network.envelope; package eu.stratosphere.runtime.io.network;
import java.io.IOException; import java.io.IOException;
......
...@@ -11,7 +11,7 @@ ...@@ -11,7 +11,7 @@
* specific language governing permissions and limitations under the License. * specific language governing permissions and limitations under the License.
**********************************************************************************************************************/ **********************************************************************************************************************/
package eu.stratosphere.runtime.io.network.envelope; package eu.stratosphere.runtime.io.network;
import java.net.InetAddress; import java.net.InetAddress;
...@@ -20,7 +20,7 @@ import eu.stratosphere.runtime.io.network.ConnectionInfoLookupResponse; ...@@ -20,7 +20,7 @@ import eu.stratosphere.runtime.io.network.ConnectionInfoLookupResponse;
import eu.stratosphere.runtime.io.network.RemoteReceiver; import eu.stratosphere.runtime.io.network.RemoteReceiver;
/** /**
* A transfer envelope receiver list contains all recipients of a transfer envelope. Their are three d ifferent types of * A transfer envelope receiver list contains all recipients of a transfer envelope. Their are three different types of
* receivers: Local receivers identified by {@link ChannelID} objects, remote receivers identified by * receivers: Local receivers identified by {@link ChannelID} objects, remote receivers identified by
* {@link InetAddress} objects and finally checkpoints which are identified by * {@link InetAddress} objects and finally checkpoints which are identified by
* <p> * <p>
......
/***********************************************************************************************************************
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network;
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.configuration.GlobalConfiguration;
import eu.stratosphere.runtime.io.network.envelope.Envelope;
import eu.stratosphere.runtime.io.network.tcp.IncomingConnectionThread;
import eu.stratosphere.runtime.io.network.tcp.OutgoingConnection;
import eu.stratosphere.runtime.io.network.tcp.OutgoingConnectionThread;
import java.io.IOException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
/**
* The network connection manager manages incoming and outgoing network connection from and to other hosts.
* <p>
* This class is thread-safe.
*
*/
public final class NetworkConnectionManager {
/**
* The default number of threads dealing with outgoing connections.
*/
private static final int DEFAULT_NUMBER_OF_OUTGOING_CONNECTION_THREADS = 1;
/**
* The default number of connection retries before giving up.
*/
private static final int DEFAULT_NUMBER_OF_CONNECTION_RETRIES = 10;
/**
* List of active threads dealing with outgoing connections.
*/
private final List<OutgoingConnectionThread> outgoingConnectionThreads = new CopyOnWriteArrayList<OutgoingConnectionThread>();
/**
* Thread dealing with incoming connections.
*/
private final IncomingConnectionThread incomingConnectionThread;
/**
* Map containing currently active outgoing connections.
*/
private final ConcurrentMap<RemoteReceiver, OutgoingConnection> outgoingConnections = new ConcurrentHashMap<RemoteReceiver, OutgoingConnection>();
/**
* The number of connection retries before giving up.
*/
private final int numberOfConnectionRetries;
/**
* A buffer provider for read buffers
*/
private final ChannelManager channelManager;
public NetworkConnectionManager(final ChannelManager channelManager,
final InetAddress bindAddress, final int dataPort) throws IOException {
final Configuration configuration = GlobalConfiguration.getConfiguration();
this.channelManager = channelManager;
// Start the connection threads
final int numberOfOutgoingConnectionThreads = configuration.getInteger(
"channel.network.numberOfOutgoingConnectionThreads", DEFAULT_NUMBER_OF_OUTGOING_CONNECTION_THREADS);
for (int i = 0; i < numberOfOutgoingConnectionThreads; i++) {
final OutgoingConnectionThread outgoingConnectionThread = new OutgoingConnectionThread();
outgoingConnectionThread.start();
this.outgoingConnectionThreads.add(outgoingConnectionThread);
}
this.incomingConnectionThread = new IncomingConnectionThread(
this.channelManager, true, new InetSocketAddress(bindAddress, dataPort));
this.incomingConnectionThread.start();
this.numberOfConnectionRetries = configuration.getInteger("channel.network.numberOfConnectionRetries",
DEFAULT_NUMBER_OF_CONNECTION_RETRIES);
}
/**
* Randomly selects one of the active threads dealing with outgoing connections.
*
* @return one of the active threads dealing with outgoing connections
*/
private OutgoingConnectionThread getOutgoingConnectionThread() {
return this.outgoingConnectionThreads.get((int) (this.outgoingConnectionThreads.size() * Math.random()));
}
/**
* Queues an envelope for transfer to a particular target host.
*
* @param remoteReceiver
* the address of the remote receiver
* @param envelope
* the envelope to be transfered
*/
public void queueEnvelopeForTransfer(final RemoteReceiver remoteReceiver, final Envelope envelope) {
getOutgoingConnection(remoteReceiver).queueEnvelope(envelope);
}
/**
* Returns (and possibly creates) the outgoing connection for the given target address.
*
* @param targetAddress
* the address of the connection target
* @return the outgoing connection object
*/
private OutgoingConnection getOutgoingConnection(final RemoteReceiver remoteReceiver) {
OutgoingConnection outgoingConnection = this.outgoingConnections.get(remoteReceiver);
if (outgoingConnection == null) {
outgoingConnection = new OutgoingConnection(remoteReceiver, getOutgoingConnectionThread(),
this.numberOfConnectionRetries);
final OutgoingConnection oldEntry = this.outgoingConnections
.putIfAbsent(remoteReceiver, outgoingConnection);
// We had a race, use the old value
if (oldEntry != null) {
outgoingConnection = oldEntry;
}
}
return outgoingConnection;
}
public void shutDown() {
// Interrupt the threads we started
this.incomingConnectionThread.interrupt();
final Iterator<OutgoingConnectionThread> it = this.outgoingConnectionThreads.iterator();
while (it.hasNext()) {
it.next().interrupt();
}
}
public void logBufferUtilization() {
System.out.println("\tOutgoing connections:");
final Iterator<Map.Entry<RemoteReceiver, OutgoingConnection>> it = this.outgoingConnections.entrySet()
.iterator();
while (it.hasNext()) {
final Map.Entry<RemoteReceiver, OutgoingConnection> entry = it.next();
System.out.println("\t\tOC " + entry.getKey() + ": " + entry.getValue().getNumberOfQueuedWriteBuffers());
}
}
}
...@@ -21,7 +21,6 @@ import java.util.List; ...@@ -21,7 +21,6 @@ import java.util.List;
import eu.stratosphere.nephele.event.task.AbstractEvent; import eu.stratosphere.nephele.event.task.AbstractEvent;
import eu.stratosphere.runtime.io.channels.ChannelID; import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.runtime.io.network.envelope.Envelope;
public final class SenderHintEvent extends AbstractEvent { public final class SenderHintEvent extends AbstractEvent {
......
/*********************************************************************************************************************** /***********************************************************************************************************************
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu) * Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu)
* *
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with * Licensed 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 * the License. You may obtain a copy of the License at
...@@ -13,16 +13,19 @@ ...@@ -13,16 +13,19 @@
package eu.stratosphere.runtime.io.network.bufferprovider; package eu.stratosphere.runtime.io.network.bufferprovider;
import eu.stratosphere.runtime.io.Buffer;
/** /**
* This interface must be implemented to receive a notification from a {@link BufferProvider} when an empty * This interface must be implemented to receive an asynchronous callback from
* {@link eu.stratosphere.runtime.io.Buffer} has * a {@link BufferProvider} as soon as a buffer has become available again.
* become available again.
*
*/ */
public interface BufferAvailabilityListener { public interface BufferAvailabilityListener {
/** /**
* Indicates that at least one {@link eu.stratosphere.runtime.io.Buffer} has become available again. * Returns a Buffer to the listener.
* <p/>
* Note: the listener has to adjust the size of the returned Buffer to the
* requested size manually via {@link Buffer#limitSize(int)}.
*/ */
void bufferAvailable(); void bufferAvailable(Buffer buffer) throws Exception;
} }
...@@ -65,5 +65,21 @@ public interface BufferProvider { ...@@ -65,5 +65,21 @@ public interface BufferProvider {
* @return <code>true</code> if the registration has been successful; <code>false</code> if the registration * @return <code>true</code> if the registration has been successful; <code>false</code> if the registration
* failed, because the buffer pool was not empty or has already been destroyed * failed, because the buffer pool was not empty or has already been destroyed
*/ */
boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener); BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener);
public enum BufferAvailabilityRegistration {
NOT_REGISTERED_BUFFER_AVAILABLE(false),
NOT_REGISTERED_BUFFER_POOL_DESTROYED(false),
REGISTERED(true);
private final boolean isSuccessful;
private BufferAvailabilityRegistration(boolean isSuccessful) {
this.isSuccessful = isSuccessful;
}
public boolean isSuccessful() {
return isSuccessful;
}
}
} }
...@@ -17,49 +17,21 @@ import eu.stratosphere.core.memory.MemorySegment; ...@@ -17,49 +17,21 @@ import eu.stratosphere.core.memory.MemorySegment;
import eu.stratosphere.runtime.io.Buffer; import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.BufferRecycler; import eu.stratosphere.runtime.io.BufferRecycler;
/** public final class DiscardBufferPool implements BufferProvider, BufferRecycler {
*
*/
public final class SerialSingleBufferPool implements BufferProvider, BufferRecycler {
private final Buffer buffer;
/** Size of the buffer in this pool */
private final int bufferSize;
// -----------------------------------------------------------------------------------------------------------------
public SerialSingleBufferPool(int bufferSize) {
this.buffer = new Buffer(new MemorySegment(new byte[bufferSize]), bufferSize, this);
this.bufferSize = bufferSize;
}
// -----------------------------------------------------------------------------------------------------------------
@Override @Override
public Buffer requestBuffer(int minBufferSize) { public Buffer requestBuffer(int minBufferSize) {
if (minBufferSize <= this.bufferSize) { return null;
return this.buffer.duplicate();
}
else {
throw new IllegalArgumentException("Requesting buffer with size " + minBufferSize + ". Pool's buffer size is " + this.bufferSize);
}
} }
@Override @Override
public Buffer requestBufferBlocking(int minBufferSize) { public Buffer requestBufferBlocking(int minBufferSize) {
if (minBufferSize <= this.bufferSize) { return null;
return this.buffer.duplicate();
}
else {
throw new IllegalArgumentException("Requesting buffer with size " + minBufferSize + ". Pool's buffer size is " + this.bufferSize);
}
} }
@Override @Override
public int getBufferSize() { public int getBufferSize() {
return this.bufferSize; return 0;
} }
@Override @Override
...@@ -68,10 +40,12 @@ public final class SerialSingleBufferPool implements BufferProvider, BufferRecyc ...@@ -68,10 +40,12 @@ public final class SerialSingleBufferPool implements BufferProvider, BufferRecyc
} }
@Override @Override
public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) { public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
throw new UnsupportedOperationException(); return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
} }
@Override @Override
public void recycle(MemorySegment buffer) {} public void recycle(MemorySegment buffer) {
throw new UnsupportedOperationException();
}
} }
...@@ -184,20 +184,20 @@ public final class LocalBufferPool implements BufferProvider { ...@@ -184,20 +184,20 @@ public final class LocalBufferPool implements BufferProvider {
} }
@Override @Override
public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) { public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
synchronized (this.buffers) { synchronized (this.buffers) {
if (!this.buffers.isEmpty()) { if (!this.buffers.isEmpty()) {
return false; return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_AVAILABLE;
} }
if (this.isDestroyed) { if (this.isDestroyed) {
return false; return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
} }
this.listeners.add(listener); this.listeners.add(listener);
} }
return true; return BufferAvailabilityRegistration.REGISTERED;
} }
/** /**
...@@ -221,7 +221,7 @@ public final class LocalBufferPool implements BufferProvider { ...@@ -221,7 +221,7 @@ public final class LocalBufferPool implements BufferProvider {
} }
this.globalBufferPool.returnBuffer(this.buffers.poll()); this.globalBufferPool.returnBuffer(this.buffers.poll());
this.numRequestedBuffers --; this.numRequestedBuffers--;
} }
this.buffers.notify(); this.buffers.notify();
...@@ -293,12 +293,18 @@ public final class LocalBufferPool implements BufferProvider { ...@@ -293,12 +293,18 @@ public final class LocalBufferPool implements BufferProvider {
if (this.isDestroyed) { if (this.isDestroyed) {
this.globalBufferPool.returnBuffer(buffer); this.globalBufferPool.returnBuffer(buffer);
this.numRequestedBuffers--; this.numRequestedBuffers--;
} else {
if (!this.listeners.isEmpty()) {
Buffer availableBuffer = new Buffer(buffer, buffer.size(), this.recycler);
try {
this.listeners.poll().bufferAvailable(availableBuffer);
} catch (Exception e) {
this.buffers.add(buffer);
this.buffers.notify();
}
} else { } else {
this.buffers.add(buffer); this.buffers.add(buffer);
this.buffers.notify(); this.buffers.notify();
while (!this.listeners.isEmpty()) {
this.listeners.poll().bufferAvailable();
} }
} }
} }
......
/***********************************************************************************************************************
* Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network.envelope;
import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProviderBroker;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.channels.ReadableByteChannel;
public class EnvelopeReader {
public enum DeserializationState {
COMPLETE,
PENDING,
NO_BUFFER_AVAILABLE;
}
private final BufferProviderBroker bufferProviderBroker;
private final ByteBuffer headerBuffer;
private ByteBuffer currentHeaderBuffer;
private ByteBuffer currentEventsList;
private ByteBuffer currentDataBuffer;
private int bufferRequestPendingWithSize;
private Envelope pendingEnvelope;
private Envelope constructedEnvelope;
public BufferProvider bufferProvider;
private JobID lastDeserializedJobID;
private ChannelID lastDeserializedSourceID;
public EnvelopeReader(BufferProviderBroker bufferProviderBroker) {
this.bufferProviderBroker = bufferProviderBroker;
this.headerBuffer = ByteBuffer.allocateDirect(EnvelopeWriter.HEADER_SIZE);
this.headerBuffer.order(ByteOrder.LITTLE_ENDIAN);
this.currentHeaderBuffer = this.headerBuffer;
}
public DeserializationState readNextChunk(ReadableByteChannel channel) throws IOException {
// 1) check if the header is pending
if (this.currentHeaderBuffer != null) {
ByteBuffer header = this.currentHeaderBuffer;
channel.read(header);
if (header.hasRemaining()) {
// not finished with the header
return DeserializationState.PENDING;
} else {
// header done, construct the envelope
this.currentHeaderBuffer = null;
Envelope env = constructEnvelopeFromHeader(header);
this.pendingEnvelope = env;
// check for events and data
int eventsSize = getEventListSize(header);
int bufferSize = getBufferSize(header);
// make the events list the next buffer to be read
if (eventsSize > 0) {
this.currentEventsList = ByteBuffer.allocate(eventsSize);
}
// if we have a data buffer, we need memory segment for it
// we may not immediately get the memory segment, though, so we first record
// that we need it
if (bufferSize > 0) {
this.bufferRequestPendingWithSize = bufferSize;
}
}
}
// 2) read the eventList, if it should have one
if (this.currentEventsList != null) {
channel.read(this.currentEventsList);
if (this.currentEventsList.hasRemaining()) {
// events list still incomplete
return DeserializationState.PENDING;
} else {
this.currentEventsList.flip();
this.pendingEnvelope.setEventsSerialized(this.currentEventsList);
this.currentEventsList = null;
}
}
// 3) check if we need to get a buffer
if (this.bufferRequestPendingWithSize > 0) {
Buffer b = getBufferForTarget(this.pendingEnvelope.getJobID(), this.pendingEnvelope.getSource(), this.bufferRequestPendingWithSize);
if (b == null) {
// no buffer available at this time. come back later
return DeserializationState.NO_BUFFER_AVAILABLE;
} else {
// buffer is available. set the field so the buffer will be filled
this.pendingEnvelope.setBuffer(b);
this.currentDataBuffer = b.getMemorySegment().wrap(0, this.bufferRequestPendingWithSize);
this.bufferRequestPendingWithSize = 0;
}
}
// 4) fill the buffer
if (this.currentDataBuffer != null) {
channel.read(this.currentDataBuffer);
if (this.currentDataBuffer.hasRemaining()) {
// data buffer incomplete
return DeserializationState.PENDING;
} else {
this.currentDataBuffer = null;
}
}
// if we get here, we completed our job, or did nothing, if the deserializer was not
// reset after the previous envelope
if (this.pendingEnvelope != null) {
this.constructedEnvelope = this.pendingEnvelope;
this.pendingEnvelope = null;
return DeserializationState.COMPLETE;
} else {
throw new IllegalStateException("Error: read() was called before reserializer was reset after the last envelope.");
}
}
private Envelope constructEnvelopeFromHeader(ByteBuffer header) throws IOException {
int magicNumber = header.getInt(EnvelopeWriter.MAGIC_NUMBER_OFFSET);
if (magicNumber != EnvelopeWriter.MAGIC_NUMBER) {
throw new IOException("Network stream corrupted: invalid magic number in envelope header.");
}
int seqNum = header.getInt(EnvelopeWriter.SEQUENCE_NUMBER_OFFSET);
JobID jid = JobID.fromByteBuffer(header, EnvelopeWriter.JOB_ID_OFFSET);
ChannelID cid = ChannelID.fromByteBuffer(header, EnvelopeWriter.CHANNEL_ID_OFFSET);
return new Envelope(seqNum, jid, cid);
}
private int getBufferSize(ByteBuffer header) {
return header.getInt(EnvelopeWriter.BUFFER_SIZE_OFFSET);
}
private int getEventListSize(ByteBuffer header) {
return header.getInt(EnvelopeWriter.EVENTS_SIZE_OFFSET);
}
private Buffer getBufferForTarget(JobID jid, ChannelID cid, int size) throws IOException {
if (!(jid.equals(this.lastDeserializedJobID) && cid.equals(this.lastDeserializedSourceID))) {
this.bufferProvider = this.bufferProviderBroker.getBufferProvider(jid, cid);
this.lastDeserializedJobID = jid;
this.lastDeserializedSourceID = cid;
}
return this.bufferProvider.requestBuffer(size);
}
public Envelope getFullyDeserializedTransferEnvelope() {
Envelope t = this.constructedEnvelope;
if (t == null) {
throw new IllegalStateException("Envelope has not yet been fully constructed.");
}
this.constructedEnvelope = null;
return t;
}
public void reset() {
this.headerBuffer.clear();
this.currentHeaderBuffer = this.headerBuffer;
this.constructedEnvelope = null;
}
public boolean hasUnfinishedData() {
return this.pendingEnvelope != null || this.currentHeaderBuffer != null;
}
public BufferProvider getBufferProvider() {
return bufferProvider;
}
public Envelope getPendingEnvelope() {
return pendingEnvelope;
}
}
/***********************************************************************************************************************
* Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network.envelope;
import eu.stratosphere.nephele.AbstractID;
import eu.stratosphere.runtime.io.Buffer;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.channels.WritableByteChannel;
public class EnvelopeWriter {
protected static final int MAGIC_NUMBER = 0xBADC0FFE;
/**
* Size of the envelope header: 48 bytes = 4 bytes magic number, 4 bytes sequence number, 16 bytes job id,
* 16 bytes sender id, 4 bytes bufferSize, 4 bytes event list length
*/
public static final int HEADER_SIZE = 4 + 4 + 2 * AbstractID.SIZE + 4 + 4;
public static final int MAGIC_NUMBER_OFFSET = 0;
public static final int SEQUENCE_NUMBER_OFFSET = 4;
public static final int JOB_ID_OFFSET = 8;
public static final int CHANNEL_ID_OFFSET = 24;
public static final int BUFFER_SIZE_OFFSET = 40;
public static final int EVENTS_SIZE_OFFSET = 44;
private ByteBuffer currentHeader;
private ByteBuffer currentEvents;
private ByteBuffer currentDataBuffer;
private final ByteBuffer headerBuffer;
public EnvelopeWriter() {
this.headerBuffer = ByteBuffer.allocateDirect(HEADER_SIZE);
this.headerBuffer.order(ByteOrder.LITTLE_ENDIAN);
}
/**
* @param channel
* @return True, if the writer has more pending data for the current envelope, false if not.
*
* @throws java.io.IOException
*/
public boolean writeNextChunk(WritableByteChannel channel) throws IOException {
// 1) check if the the header is still pending
if (this.currentHeader != null) {
channel.write(this.currentHeader);
if (this.currentHeader.hasRemaining()) {
// header was not fully written, so we can leave this method
return true;
} else {
this.currentHeader = null;
}
}
// 2) check if there are events pending
if (this.currentEvents != null) {
channel.write(this.currentEvents);
if (this.currentEvents.hasRemaining()) {
// events were not fully written, so leave this method
return true;
} else {
this.currentEvents = null;
}
}
// 3) write the data buffer
if (this.currentDataBuffer != null) {
channel.write(this.currentDataBuffer);
if (this.currentDataBuffer.hasRemaining()) {
return true;
} else {
this.currentDataBuffer = null;
}
}
return false;
}
public void setEnvelopeForWriting(Envelope env) {
// header
constructHeader(env);
this.currentHeader = this.headerBuffer;
// events (possibly null)
this.currentEvents = env.getEventsSerialized();
// data buffer (possibly null)
Buffer buf = env.getBuffer();
if (buf != null && buf.size() > 0) {
this.currentDataBuffer = buf.getMemorySegment().wrap(0, buf.size());
}
}
private void constructHeader(Envelope env) {
final ByteBuffer buf = this.headerBuffer;
buf.clear(); // reset
buf.putInt(MAGIC_NUMBER);
buf.putInt(env.getSequenceNumber()); // sequence number (4 bytes)
env.getJobID().write(buf); // job Id (16 bytes)
env.getSource().write(buf); // producerId (16 bytes)
// buffer size
buf.putInt(env.getBuffer() == null ? 0 : env.getBuffer().size());
// size of event list
buf.putInt(env.getEventsSerialized() == null ? 0 : env.getEventsSerialized().remaining());
buf.flip();
}
}
/***********************************************************************************************************************
* Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network.netty;
import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProviderBroker;
import eu.stratosphere.runtime.io.network.Envelope;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.concurrent.ConcurrentLinkedQueue;
public class InboundEnvelopeDecoder extends ChannelInboundHandlerAdapter implements BufferAvailabilityListener {
private final BufferProviderBroker bufferProviderBroker;
private final BufferAvailabilityChangedTask bufferAvailabilityChangedTask = new BufferAvailabilityChangedTask();
private final ConcurrentLinkedQueue<Buffer> bufferBroker = new ConcurrentLinkedQueue<Buffer>();
private final ByteBuffer headerBuffer;
private Envelope currentEnvelope;
private ByteBuffer currentEventsBuffer;
private ByteBuffer currentDataBuffer;
private int currentBufferRequestSize;
private BufferProvider currentBufferProvider;
private JobID lastJobId;
private ChannelID lastSourceId;
private ByteBuf stagedBuffer;
private ChannelHandlerContext channelHandlerContext;
private int bytesToSkip;
private enum DecoderState {
COMPLETE,
PENDING,
NO_BUFFER_AVAILABLE
}
public InboundEnvelopeDecoder(BufferProviderBroker bufferProviderBroker) {
this.bufferProviderBroker = bufferProviderBroker;
this.headerBuffer = ByteBuffer.allocateDirect(OutboundEnvelopeEncoder.HEADER_SIZE);
}
@Override
public void channelActive(ChannelHandlerContext ctx) throws Exception {
if (this.channelHandlerContext == null) {
this.channelHandlerContext = ctx;
}
super.channelActive(ctx);
}
@Override
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
if (this.stagedBuffer != null) {
throw new IllegalStateException("No channel read event should be fired " +
"as long as the a buffer is staged.");
}
ByteBuf in = (ByteBuf) msg;
if (this.bytesToSkip > 0) {
this.bytesToSkip = skipBytes(in, this.bytesToSkip);
// we skipped over the whole buffer
if (this.bytesToSkip > 0) {
in.release();
return;
}
}
decodeBuffer(in, ctx);
}
/**
* Decodes all Envelopes contained in a Netty ByteBuf and forwards them in the pipeline.
* Returns true and releases the buffer, if it was fully consumed. Otherwise, returns false and retains the buffer.
* </p>
* In case of no buffer availability (returns false), a buffer availability listener is registered and the input
* buffer is staged for later consumption.
*
* @return <code>true</code>, if buffer fully consumed, <code>false</code> otherwise
* @throws IOException
*/
private boolean decodeBuffer(ByteBuf in, ChannelHandlerContext ctx) throws IOException {
DecoderState decoderState;
while ((decoderState = decodeEnvelope(in)) != DecoderState.PENDING) {
if (decoderState == DecoderState.COMPLETE) {
ctx.fireChannelRead(this.currentEnvelope);
this.currentEnvelope = null;
}
else if (decoderState == DecoderState.NO_BUFFER_AVAILABLE) {
switch (this.currentBufferProvider.registerBufferAvailabilityListener(this)) {
case REGISTERED:
if (ctx.channel().config().isAutoRead()) {
ctx.channel().config().setAutoRead(false);
}
this.stagedBuffer = in;
this.stagedBuffer.retain();
return false;
case NOT_REGISTERED_BUFFER_AVAILABLE:
continue;
case NOT_REGISTERED_BUFFER_POOL_DESTROYED:
this.bytesToSkip = skipBytes(in, this.currentBufferRequestSize);
this.currentBufferRequestSize = 0;
this.currentEventsBuffer = null;
this.currentEnvelope = null;
}
}
}
if (in.isReadable()) {
throw new IllegalStateException("Every buffer should have been fully" +
"consumed after *successfully* decoding it (if it was not successful, " +
"the buffer will be staged for later consumption).");
}
in.release();
return true;
}
/**
* Notifies the IO thread that a Buffer has become available again.
* <p/>
* This method will be called from outside the Netty IO thread. The caller will be the buffer pool from which the
* available buffer comes (i.e. the InputGate).
* <p/>
* We have to make sure that the available buffer is handed over to the IO thread in a safe manner.
*/
@Override
public void bufferAvailable(Buffer buffer) throws Exception {
this.bufferBroker.offer(buffer);
this.channelHandlerContext.channel().eventLoop().execute(this.bufferAvailabilityChangedTask);
}
/**
* Continues the decoding of a staged buffer after a buffer has become available again.
* <p/>
* This task should be executed by the IO thread to ensure safe access to the staged buffer.
*/
private class BufferAvailabilityChangedTask implements Runnable {
@Override
public void run() {
Buffer availableBuffer = bufferBroker.poll();
if (availableBuffer == null) {
throw new IllegalStateException("The BufferAvailabilityChangedTask" +
"should only be executed when a Buffer has been offered" +
"to the Buffer broker (after becoming available).");
}
// This alters the state of the last `decodeEnvelope(ByteBuf)`
// call to set the buffer, which has become available again
availableBuffer.limitSize(currentBufferRequestSize);
currentEnvelope.setBuffer(availableBuffer);
currentDataBuffer = availableBuffer.getMemorySegment().wrap(0, InboundEnvelopeDecoder.this.currentBufferRequestSize);
currentBufferRequestSize = 0;
stagedBuffer.release();
try {
if (decodeBuffer(stagedBuffer, channelHandlerContext)) {
stagedBuffer = null;
channelHandlerContext.channel().config().setAutoRead(true);
}
} catch (IOException e) {
availableBuffer.recycleBuffer();
}
}
}
// --------------------------------------------------------------------
private DecoderState decodeEnvelope(ByteBuf in) throws IOException {
// --------------------------------------------------------------------
// (1) header (EnvelopeEncoder.HEADER_SIZE bytes)
// --------------------------------------------------------------------
if (this.currentEnvelope == null) {
copy(in, this.headerBuffer);
if (this.headerBuffer.hasRemaining()) {
return DecoderState.PENDING;
}
else {
this.headerBuffer.flip();
int magicNum = this.headerBuffer.getInt();
if (magicNum != OutboundEnvelopeEncoder.MAGIC_NUMBER) {
throw new IOException("Network stream corrupted: invalid magic" +
"number in current envelope header.");
}
int seqNum = this.headerBuffer.getInt();
JobID jobId = JobID.fromByteBuffer(this.headerBuffer);
ChannelID sourceId = ChannelID.fromByteBuffer(this.headerBuffer);
this.currentEnvelope = new Envelope(seqNum, jobId, sourceId);
int eventsSize = this.headerBuffer.getInt();
int bufferSize = this.headerBuffer.getInt();
this.currentEventsBuffer = eventsSize > 0 ? ByteBuffer.allocate(eventsSize) : null;
this.currentBufferRequestSize = bufferSize > 0 ? bufferSize : 0;
this.headerBuffer.clear();
}
}
// --------------------------------------------------------------------
// (2) events (var length)
// --------------------------------------------------------------------
if (this.currentEventsBuffer != null) {
copy(in, this.currentEventsBuffer);
if (this.currentEventsBuffer.hasRemaining()) {
return DecoderState.PENDING;
}
else {
this.currentEventsBuffer.flip();
this.currentEnvelope.setEventsSerialized(this.currentEventsBuffer);
this.currentEventsBuffer = null;
}
}
// --------------------------------------------------------------------
// (3) buffer (var length)
// --------------------------------------------------------------------
// (a) request a buffer from OUR pool
if (this.currentBufferRequestSize > 0) {
JobID jobId = this.currentEnvelope.getJobID();
ChannelID sourceId = this.currentEnvelope.getSource();
Buffer buffer = requestBufferForTarget(jobId, sourceId, this.currentBufferRequestSize);
if (buffer == null) {
return DecoderState.NO_BUFFER_AVAILABLE;
}
else {
this.currentEnvelope.setBuffer(buffer);
this.currentDataBuffer = buffer.getMemorySegment().wrap(0, this.currentBufferRequestSize);
this.currentBufferRequestSize = 0;
}
}
// (b) copy data to OUR buffer
if (this.currentDataBuffer != null) {
copy(in, this.currentDataBuffer);
if (this.currentDataBuffer.hasRemaining()) {
return DecoderState.PENDING;
}
else {
this.currentDataBuffer = null;
}
}
// if we made it to this point, we completed the envelope;
// in the other cases we return early with PENDING or NO_BUFFER_AVAILABLE
return DecoderState.COMPLETE;
}
private Buffer requestBufferForTarget(JobID jobId, ChannelID sourceId, int size) throws IOException {
// Request the buffer from the target buffer provider, which is the
// InputGate of the receiving InputChannel.
if (!(jobId.equals(this.lastJobId) && sourceId.equals(this.lastSourceId))) {
this.lastJobId = jobId;
this.lastSourceId = sourceId;
this.currentBufferProvider = this.bufferProviderBroker.getBufferProvider(jobId, sourceId);
}
return this.currentBufferProvider.requestBuffer(size);
}
/**
* Copies min(from.readableBytes(), to.remaining() bytes from Nettys ByteBuf to the Java NIO ByteBuffer.
*/
private void copy(ByteBuf src, ByteBuffer dst) {
// This branch is necessary, because an Exception is thrown if the
// destination buffer has more remaining (writable) bytes than
// currently readable from the Netty ByteBuf source.
if (src.isReadable()) {
if (src.readableBytes() < dst.remaining()) {
int oldLimit = dst.limit();
dst.limit(dst.position() + src.readableBytes());
src.readBytes(dst);
dst.limit(oldLimit);
}
else {
src.readBytes(dst);
}
}
}
/**
* Skips over min(in.readableBytes(), toSkip) bytes in the Netty ByteBuf and returns how many bytes remain to be
* skipped.
*
* @return remaining bytes to be skipped
*/
private int skipBytes(ByteBuf in, int toSkip) {
if (toSkip <= in.readableBytes()) {
in.readBytes(toSkip);
return 0;
}
int remainingToSkip = toSkip - in.readableBytes();
in.readerIndex(in.readerIndex() + in.readableBytes());
return remainingToSkip;
}
}
/***********************************************************************************************************************
* Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network.netty;
import eu.stratosphere.runtime.io.network.Envelope;
import eu.stratosphere.runtime.io.network.EnvelopeDispatcher;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
public class InboundEnvelopeDispatcherHandler extends ChannelInboundHandlerAdapter {
private static final Log LOG = LogFactory.getLog(InboundEnvelopeDispatcherHandler.class);
private final EnvelopeDispatcher channelManager;
public InboundEnvelopeDispatcherHandler(EnvelopeDispatcher channelManager) {
this.channelManager = channelManager;
}
@Override
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
Envelope envelope = (Envelope) msg;
// LOG.debug(String.format("Decoded envelope with seq num %d from source channel %s",
// envelope.getSequenceNumber(),
// envelope.getSource()));
this.channelManager.dispatchFromNetwork(envelope);
}
}
/***********************************************************************************************************************
* Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network.netty;
import eu.stratosphere.runtime.io.network.ChannelManager;
import eu.stratosphere.runtime.io.network.Envelope;
import eu.stratosphere.runtime.io.network.RemoteReceiver;
import io.netty.bootstrap.Bootstrap;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.PooledByteBufAllocator;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.FixedRecvByteBufAllocator;
import io.netty.channel.nio.NioEventLoopGroup;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
import io.netty.util.concurrent.Future;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.io.IOException;
import java.net.InetAddress;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
public class NettyConnectionManager {
private static final Log LOG = LogFactory.getLog(NettyConnectionManager.class);
private final ChannelManager channelManager;
private final ServerBootstrap in;
private final Bootstrap out;
private final ConcurrentMap<RemoteReceiver, Object> outConnections;
public NettyConnectionManager(ChannelManager channelManager, InetAddress bindAddress, int bindPort,
int bufferSize, int numInThreads, int numOutThreads,
int lowWaterMark, int highWaterMark) {
this.outConnections = new ConcurrentHashMap<RemoteReceiver, Object>();
this.channelManager = channelManager;
// --------------------------------------------------------------------
int defaultNumThreads = Math.max(Runtime.getRuntime().availableProcessors() / 4, 1);
numInThreads = (numInThreads == -1) ? defaultNumThreads : numInThreads;
numOutThreads = (numOutThreads == -1) ? defaultNumThreads : numOutThreads;
LOG.info(String.format("Starting with %d incoming and %d outgoing connection threads.", numInThreads, numOutThreads));
lowWaterMark = (lowWaterMark == -1) ? bufferSize / 2 : lowWaterMark;
highWaterMark = (highWaterMark == -1) ? bufferSize : highWaterMark;
LOG.info(String.format("Setting low water mark to %d and high water mark to %d bytes.", lowWaterMark, highWaterMark));
// --------------------------------------------------------------------
// server bootstrap (incoming connections)
// --------------------------------------------------------------------
this.in = new ServerBootstrap();
this.in.group(new NioEventLoopGroup(numInThreads))
.channel(NioServerSocketChannel.class)
.localAddress(bindAddress, bindPort)
.childHandler(new ChannelInitializer<SocketChannel>() {
@Override
public void initChannel(SocketChannel channel) throws Exception {
channel.pipeline()
.addLast(new InboundEnvelopeDecoder(NettyConnectionManager.this.channelManager))
.addLast(new InboundEnvelopeDispatcherHandler(NettyConnectionManager.this.channelManager));
}
})
.option(ChannelOption.RCVBUF_ALLOCATOR, new FixedRecvByteBufAllocator(bufferSize))
.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT);
// --------------------------------------------------------------------
// client bootstrap (outgoing connections)
// --------------------------------------------------------------------
this.out = new Bootstrap();
this.out.group(new NioEventLoopGroup(numOutThreads))
.channel(NioSocketChannel.class)
.handler(new ChannelInitializer<SocketChannel>() {
@Override
public void initChannel(SocketChannel channel) throws Exception {
channel.pipeline()
.addLast(new OutboundEnvelopeEncoder());
}
})
.option(ChannelOption.WRITE_BUFFER_LOW_WATER_MARK, lowWaterMark)
.option(ChannelOption.WRITE_BUFFER_HIGH_WATER_MARK, highWaterMark)
.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT)
.option(ChannelOption.TCP_NODELAY, false)
.option(ChannelOption.SO_KEEPALIVE, true);
try {
this.in.bind().sync();
} catch (InterruptedException e) {
throw new RuntimeException("Could not bind server socket for incoming connections.");
}
}
public void shutdown() {
Future<?> inShutdownFuture = this.in.group().shutdownGracefully();
Future<?> outShutdownFuture = this.out.group().shutdownGracefully();
try {
inShutdownFuture.sync();
outShutdownFuture.sync();
} catch (InterruptedException e) {
throw new RuntimeException("Could not properly shutdown connections.");
}
}
public void enqueue(Envelope envelope, RemoteReceiver receiver) throws IOException {
// Get the channel. The channel may be
// 1) a channel that already exists (usual case) -> just send the data
// 2) a channel that is in buildup (sometimes) -> attach to the future and wait for the actual channel
// 3) not yet existing -> establish the channel
final Object entry = this.outConnections.get(receiver);
final OutboundConnectionQueue channel;
if (entry != null) {
// existing channel or channel in buildup
if (entry instanceof OutboundConnectionQueue) {
channel = (OutboundConnectionQueue) entry;
}
else {
ChannelInBuildup future = (ChannelInBuildup) entry;
channel = future.waitForChannel();
}
}
else {
// No channel yet. Create one, but watch out for a race.
// We create a "buildup future" and atomically add it to the map.
// Only the thread that really added it establishes the channel.
// The others need to wait on that original establisher's future.
ChannelInBuildup inBuildup = new ChannelInBuildup(this.out, receiver);
Object old = this.outConnections.putIfAbsent(receiver, inBuildup);
if (old == null) {
this.out.connect(receiver.getConnectionAddress()).addListener(inBuildup);
channel = inBuildup.waitForChannel();
Object previous = this.outConnections.put(receiver, channel);
if (inBuildup != previous) {
throw new IOException("Race condition during channel build up.");
}
}
else if (old instanceof ChannelInBuildup) {
channel = ((ChannelInBuildup) old).waitForChannel();
}
else {
channel = (OutboundConnectionQueue) old;
}
}
channel.enqueue(envelope);
}
// ------------------------------------------------------------------------
private static final class ChannelInBuildup implements ChannelFutureListener {
private Object lock = new Object();
private volatile OutboundConnectionQueue channel;
private volatile Throwable error;
private int numRetries = 2;
private final Bootstrap out;
private final RemoteReceiver receiver;
private ChannelInBuildup(Bootstrap out, RemoteReceiver receiver) {
this.out = out;
this.receiver = receiver;
}
private void handInChannel(OutboundConnectionQueue c) {
synchronized (this.lock) {
this.channel = c;
this.lock.notifyAll();
}
}
private void notifyOfError(Throwable error) {
synchronized (this.lock) {
this.error = error;
this.lock.notifyAll();
}
}
private OutboundConnectionQueue waitForChannel() throws IOException {
synchronized (this.lock) {
while (this.error == null && this.channel == null) {
try {
this.lock.wait(2000);
} catch (InterruptedException e) {
throw new RuntimeException("Channel buildup interrupted.");
}
}
}
if (this.error != null) {
throw new IOException("Connecting the channel failed: " + error.getMessage(), error);
}
return this.channel;
}
@Override
public void operationComplete(ChannelFuture future) throws Exception {
if (future.isSuccess()) {
if (LOG.isDebugEnabled()) {
LOG.debug(String.format("Channel %s connected", future.channel()));
}
handInChannel(new OutboundConnectionQueue(future.channel()));
}
else if (this.numRetries > 0) {
LOG.debug(String.format("Connection request did not succeed, retrying (%d attempts left)", this.numRetries));
this.out.connect(this.receiver.getConnectionAddress()).addListener(this);
this.numRetries--;
}
else {
if (future.getClass() != null) {
notifyOfError(future.cause());
}
else {
notifyOfError(new Exception("Connection could not be established."));
}
}
}
}
}
/***********************************************************************************************************************
* Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network.netty;
import eu.stratosphere.runtime.io.network.Envelope;
import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.util.ArrayDeque;
public class OutboundConnectionQueue extends ChannelInboundHandlerAdapter implements ChannelFutureListener {
private static final Log LOG = LogFactory.getLog(OutboundConnectionQueue.class);
private final Channel channel;
private final ArrayDeque<Envelope> queuedEnvelopes;
public OutboundConnectionQueue(Channel channel) {
this.channel = channel;
this.queuedEnvelopes = new ArrayDeque<Envelope>();
channel.pipeline().addFirst(this);
}
/**
* Enqueues an envelope so be sent later.
* <p/>
* This method is always invoked by the task thread that wants the envelope sent.
*
* @param env The envelope to be sent.
*/
public void enqueue(Envelope env) {
// the user event trigger ensure thread-safe hand-over of the envelope
this.channel.pipeline().fireUserEventTriggered(env);
}
@Override
public void userEventTriggered(ChannelHandlerContext ctx, Object envelopeToEnqueue) throws Exception {
boolean triggerWrite = this.queuedEnvelopes.isEmpty();
this.queuedEnvelopes.addLast((Envelope) envelopeToEnqueue);
if (triggerWrite) {
writeAndFlushNextEnvelopeIfPossible();
}
}
@Override
public void channelWritabilityChanged(ChannelHandlerContext ctx) throws Exception {
writeAndFlushNextEnvelopeIfPossible();
}
@Override
public void operationComplete(ChannelFuture future) throws Exception {
if (future.isSuccess()) {
writeAndFlushNextEnvelopeIfPossible();
}
else if (future.cause() != null) {
exceptionOccurred(future.cause());
}
else {
exceptionOccurred(new Exception("Envelope send aborted."));
}
}
private void writeAndFlushNextEnvelopeIfPossible() {
if (this.channel.isWritable() && !this.queuedEnvelopes.isEmpty()) {
Envelope nextEnvelope = this.queuedEnvelopes.pollFirst();
this.channel.writeAndFlush(nextEnvelope).addListener(this);
}
}
private void exceptionOccurred(Throwable t) throws Exception {
LOG.error(String.format("An exception occurred in Channel %s: %s", this.channel, t.getMessage()));
throw new Exception(t);
}
}
/***********************************************************************************************************************
* Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network.netty;
import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.network.Envelope;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelHandler;
import io.netty.channel.ChannelHandlerContext;
import io.netty.handler.codec.MessageToByteEncoder;
@ChannelHandler.Sharable
public class OutboundEnvelopeEncoder extends MessageToByteEncoder<Envelope> {
public static final int HEADER_SIZE = 48;
public static final int MAGIC_NUMBER = 0xBADC0FFE;
@Override
protected void encode(ChannelHandlerContext ctx, Envelope env, ByteBuf out) throws Exception {
// --------------------------------------------------------------------
// (1) header (48 bytes)
// --------------------------------------------------------------------
out.writeInt(MAGIC_NUMBER); // 4 bytes
if (out.getInt(out.writerIndex()-4) != MAGIC_NUMBER) {
throw new RuntimeException();
}
out.writeInt(env.getSequenceNumber()); // 4 bytes
env.getJobID().writeTo(out); // 16 bytes
env.getSource().writeTo(out); // 16 bytes
out.writeInt(env.getEventsSerialized() != null ? env.getEventsSerialized().remaining() : 0); // 4 bytes
out.writeInt(env.getBuffer() != null ? env.getBuffer().size() : 0); // 4 bytes
// --------------------------------------------------------------------
// (2) events (var length)
// --------------------------------------------------------------------
if (env.getEventsSerialized() != null) {
out.writeBytes(env.getEventsSerialized());
}
// --------------------------------------------------------------------
// (3) buffer (var length)
// --------------------------------------------------------------------
if (env.getBuffer() != null) {
Buffer buffer = env.getBuffer();
out.writeBytes(buffer.getMemorySegment().wrap(0, buffer.size()));
// Recycle the buffer from OUR buffer pool after everything has been
// copied to Nettys buffer space.
buffer.recycleBuffer();
}
}
}
/***********************************************************************************************************************
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network.tcp;
import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.network.ChannelManager;
import eu.stratosphere.runtime.io.network.envelope.Envelope;
import eu.stratosphere.runtime.io.network.envelope.EnvelopeReader;
import eu.stratosphere.runtime.io.network.envelope.EnvelopeReader.DeserializationState;
import eu.stratosphere.runtime.io.network.envelope.NoBufferAvailableException;
import eu.stratosphere.util.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.io.IOException;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.SelectionKey;
/**
* An incoming TCP connection through which data is read and transformed into {@link Envelope} objects.
*/
public class IncomingConnection {
private static final Log LOG = LogFactory.getLog(IncomingConnection.class);
/** Readable byte channel (TCP socket) to read data from */
private final ReadableByteChannel channel;
/** Channel manager to dispatch complete envelopes */
private final ChannelManager channelManager;
/** Envelope reader to turn the channel data into envelopes */
private final EnvelopeReader reader;
// -----------------------------------------------------------------------------------------------------------------
public IncomingConnection(ReadableByteChannel channel, ChannelManager channelManager) {
this.channel = channel;
this.channelManager = channelManager;
this.reader = new EnvelopeReader(channelManager);
}
// -----------------------------------------------------------------------------------------------------------------
public void read() throws IOException, InterruptedException, NoBufferAvailableException {
DeserializationState deserializationState = this.reader.readNextChunk(this.channel);
switch (deserializationState) {
case COMPLETE:
Envelope envelope = this.reader.getFullyDeserializedTransferEnvelope();
this.channelManager.dispatchFromNetwork(envelope);
this.reader.reset();
break;
case NO_BUFFER_AVAILABLE:
throw new NoBufferAvailableException(this.reader.getBufferProvider());
case PENDING:
break;
}
}
public void reportTransmissionProblem(SelectionKey key, IOException ioe) {
LOG.error(StringUtils.stringifyException(ioe));
try {
this.channel.close();
} catch (IOException e) {
LOG.debug("An error occurred while closing the byte channel");
}
if (key != null) {
key.cancel();
}
Envelope pendingEnvelope = this.reader.getPendingEnvelope();
if (pendingEnvelope != null) {
if (pendingEnvelope.hasBuffer()) {
Buffer buffer = pendingEnvelope.getBuffer();
if (buffer != null) {
buffer.recycleBuffer();
}
}
}
this.reader.reset();
}
public boolean isCloseUnexpected() {
return this.reader.hasUnfinishedData();
}
public void closeConnection(SelectionKey key) {
try {
this.channel.close();
} catch (IOException ioe) {
LOG.error("An IOException occurred while closing the socket: + " + StringUtils.stringifyException(ioe));
}
if (key != null) {
key.cancel();
}
}
}
/***********************************************************************************************************************
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network.tcp;
import eu.stratosphere.runtime.io.network.ChannelManager;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
import eu.stratosphere.runtime.io.network.envelope.NoBufferAvailableException;
import eu.stratosphere.util.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.io.EOFException;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.nio.channels.ClosedChannelException;
import java.nio.channels.SelectionKey;
import java.nio.channels.Selector;
import java.nio.channels.ServerSocketChannel;
import java.nio.channels.SocketChannel;
import java.util.ArrayDeque;
import java.util.Iterator;
import java.util.Queue;
public class IncomingConnectionThread extends Thread {
private static final Log LOG = LogFactory.getLog(IncomingConnectionThread.class);
private final ChannelManager channelManager;
private final Selector selector;
private final Queue<SelectionKey> pendingReadEventSubscribeRequests = new ArrayDeque<SelectionKey>();
private final ServerSocketChannel listeningSocket;
private static final class IncomingConnectionBufferAvailListener implements BufferAvailabilityListener {
private final Queue<SelectionKey> pendingReadEventSubscribeRequests;
private final SelectionKey key;
private IncomingConnectionBufferAvailListener(final Queue<SelectionKey> pendingReadEventSubscribeRequests,
final SelectionKey key) {
this.pendingReadEventSubscribeRequests = pendingReadEventSubscribeRequests;
this.key = key;
}
@Override
public void bufferAvailable() {
synchronized (this.pendingReadEventSubscribeRequests) {
this.pendingReadEventSubscribeRequests.add(this.key);
}
}
}
public IncomingConnectionThread(ChannelManager channelManager,
boolean isListeningThread, InetSocketAddress listeningAddress) throws IOException {
super("Incoming Connection Thread");
this.selector = Selector.open();
this.channelManager = channelManager;
if (isListeningThread) {
this.listeningSocket = ServerSocketChannel.open();
this.listeningSocket.configureBlocking(false);
listeningSocket.register(this.selector, SelectionKey.OP_ACCEPT);
this.listeningSocket.socket().bind(listeningAddress);
LOG.debug("Listening on " + this.listeningSocket.socket().getLocalSocketAddress());
} else {
this.listeningSocket = null;
}
}
@Override
public void run() {
try {
while (!this.isInterrupted()) {
synchronized (this.pendingReadEventSubscribeRequests) {
while (!this.pendingReadEventSubscribeRequests.isEmpty()) {
final SelectionKey key = this.pendingReadEventSubscribeRequests.poll();
final IncomingConnection incomingConnection = (IncomingConnection) key.attachment();
final SocketChannel socketChannel = (SocketChannel) key.channel();
try {
final SelectionKey newKey = socketChannel.register(this.selector, SelectionKey.OP_READ);
newKey.attach(incomingConnection);
} catch (ClosedChannelException e) {
incomingConnection.reportTransmissionProblem(key, e);
}
}
}
try {
this.selector.select(500);
} catch (IOException e) {
LOG.error(e);
}
final Iterator<SelectionKey> iter = this.selector.selectedKeys().iterator();
while (iter.hasNext()) {
final SelectionKey key = iter.next();
iter.remove();
if (key.isValid()) {
if (key.isReadable()) {
doRead(key);
} else if (key.isAcceptable()) {
doAccept(key);
} else {
LOG.error("Unknown key: " + key);
}
} else {
LOG.error("Received invalid key: " + key);
}
}
}
// Do cleanup, if necessary
if (this.listeningSocket != null) {
try {
this.listeningSocket.close();
} catch (IOException ioe) {
// Actually, we can ignore this exception
LOG.debug(ioe);
}
}
// Finally, close the selector
try {
this.selector.close();
} catch (IOException ioe) {
LOG.debug(StringUtils.stringifyException(ioe));
}
}
catch (Throwable t) {
// this is a disaster, this task manager cannot go on!
LOG.fatal("Incoming connection thread died with an exception: " + t.getMessage(), t);
System.exit(1);
}
}
private void doAccept(SelectionKey key) {
SocketChannel clientSocket = null;
try {
clientSocket = this.listeningSocket.accept();
if (clientSocket == null) {
LOG.error("Client socket is null");
return;
}
} catch (IOException ioe) {
LOG.error(ioe);
return;
}
final IncomingConnection incomingConnection = new IncomingConnection(
clientSocket, this.channelManager);
SelectionKey clientKey = null;
try {
clientSocket.configureBlocking(false);
clientKey = clientSocket.register(this.selector, SelectionKey.OP_READ);
clientKey.attach(incomingConnection);
} catch (IOException ioe) {
incomingConnection.reportTransmissionProblem(clientKey, ioe);
}
}
private void doRead(SelectionKey key) {
final IncomingConnection incomingConnection = (IncomingConnection) key.attachment();
try {
incomingConnection.read();
} catch (EOFException eof) {
if (incomingConnection.isCloseUnexpected()) {
final SocketChannel socketChannel = (SocketChannel) key.channel();
LOG.error("Connection from " + socketChannel.socket().getRemoteSocketAddress()
+ " was closed unexpectedly");
incomingConnection.reportTransmissionProblem(key, eof);
} else {
incomingConnection.closeConnection(key);
}
} catch (IOException ioe) {
incomingConnection.reportTransmissionProblem(key, ioe);
} catch (InterruptedException e) {
// Nothing to do here
} catch (NoBufferAvailableException e) {
// There are no buffers available, unsubscribe from read event
final SocketChannel socketChannel = (SocketChannel) key.channel();
try {
final SelectionKey newKey = socketChannel.register(this.selector, 0);
newKey.attach(incomingConnection);
} catch (ClosedChannelException e1) {
incomingConnection.reportTransmissionProblem(key, e1);
}
final BufferAvailabilityListener bal = new IncomingConnectionBufferAvailListener(
this.pendingReadEventSubscribeRequests, key);
if (!e.getBufferProvider().registerBufferAvailabilityListener(bal)) {
// In the meantime, a buffer has become available again, subscribe to read event again
try {
final SelectionKey newKey = socketChannel.register(this.selector, SelectionKey.OP_READ);
newKey.attach(incomingConnection);
} catch (ClosedChannelException e1) {
incomingConnection.reportTransmissionProblem(key, e1);
}
}
}
}
}
/***********************************************************************************************************************
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network.tcp;
import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.runtime.io.network.RemoteReceiver;
import eu.stratosphere.runtime.io.network.envelope.Envelope;
import eu.stratosphere.runtime.io.network.envelope.EnvelopeWriter;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.nio.channels.SelectionKey;
import java.nio.channels.SocketChannel;
import java.nio.channels.WritableByteChannel;
import java.util.ArrayDeque;
import java.util.Iterator;
import java.util.Queue;
/**
* This class represents an outgoing TCP connection through which {@link eu.stratosphere.runtime.io.network.envelope.Envelope} objects can be sent.
* {@link eu.stratosphere.runtime.io.network.envelope.Envelope} objects are received from the {@link eu.stratosphere.nephele.taskmanager.io.bytebuffered.ChannelManager} and added to a queue. An
* additional network thread then takes the envelopes from the queue and transmits them to the respective destination
* host.
*
*/
public class OutgoingConnection {
/**
* The log object used to report debug information and possible errors.
*/
private static final Log LOG = LogFactory.getLog(OutgoingConnection.class);
/**
* The address this outgoing connection is connected to.
*/
private final RemoteReceiver remoteReceiver;
/**
* The outgoing connection thread which actually transmits the queued transfer envelopes.
*/
private final OutgoingConnectionThread connectionThread;
/**
* The queue of transfer envelopes to be transmitted.
*/
private final Queue<Envelope> queuedEnvelopes = new ArrayDeque<Envelope>();
/**
* The {@link eu.stratosphere.runtime.io.network.envelope.Envelope} that is currently processed.
*/
private Envelope currentEnvelope = null;
/**
* Stores whether the underlying TCP connection is established. As this variable is accessed by the byte buffered
* channel manager and the outgoing connection thread, it must be protected by a monitor.
*/
private boolean isConnected = false;
/**
* Stores whether is underlying TCP connection is subscribed to the NIO write event. As this variable is accessed by
* the byte buffered channel and the outgoing connection thread, it must be protected by a monitor.
*/
private boolean isSubscribedToWriteEvent = false;
/**
* The overall number of connection retries which shall be performed before a connection error is reported.
*/
private final int numberOfConnectionRetries;
/**
* The number of connection retries left before an I/O error is reported.
*/
private int retriesLeft = 0;
/**
* The timestamp of the last connection retry.
*/
private long timstampOfLastRetry = 0;
/**
* The current selection key representing the interest set of the underlying TCP NIO connection. This variable may
* only be accessed the the outgoing connection thread.
*/
private SelectionKey selectionKey = null;
/**
* The period of time in milliseconds that shall be waited before a connection attempt is considered to be failed.
*/
private static long RETRYINTERVAL = 1000L; // 1 second
private EnvelopeWriter writer;
/**
* Constructs a new outgoing connection object.
*
* @param remoteReceiver
* the address of the destination host this outgoing connection object is supposed to connect to
* @param connectionThread
* the connection thread which actually handles the network transfer
* @param numberOfConnectionRetries
* the number of connection retries allowed before an I/O error is reported
*/
public OutgoingConnection(RemoteReceiver remoteReceiver, OutgoingConnectionThread connectionThread,
int numberOfConnectionRetries) {
this.remoteReceiver = remoteReceiver;
this.connectionThread = connectionThread;
this.numberOfConnectionRetries = numberOfConnectionRetries;
this.writer = new EnvelopeWriter();
}
/**
* Adds a new {@link eu.stratosphere.runtime.io.network.envelope.Envelope} to the queue of envelopes to be transmitted to the destination host of this
* connection.
* <p>
* This method should only be called by the {@link eu.stratosphere.nephele.taskmanager.io.bytebuffered.ChannelManager} object.
*
* @param envelope
* the envelope to be added to the transfer queue
*/
public void queueEnvelope(Envelope envelope) {
synchronized (this.queuedEnvelopes) {
checkConnection();
this.queuedEnvelopes.add(envelope);
}
}
private void checkConnection() {
synchronized (this.queuedEnvelopes) {
if (!this.isConnected) {
this.retriesLeft = this.numberOfConnectionRetries;
this.timstampOfLastRetry = System.currentTimeMillis();
this.connectionThread.triggerConnect(this);
this.isConnected = true;
this.isSubscribedToWriteEvent = true;
} else {
if (!this.isSubscribedToWriteEvent) {
this.connectionThread.subscribeToWriteEvent(this.selectionKey);
this.isSubscribedToWriteEvent = true;
}
}
}
}
/**
* Returns the {@link InetSocketAddress} to the destination host this outgoing connection is supposed to be
* connected to.
* <p>
* This method should be called by the {@link OutgoingConnectionThread} object only.
*
* @return the {@link InetSocketAddress} to the destination host this outgoing connection is supposed to be
* connected to
*/
public InetSocketAddress getConnectionAddress() {
return this.remoteReceiver.getConnectionAddress();
}
/**
* Reports a problem which occurred while establishing the underlying TCP connection to this outgoing connection
* object. Depending on the number of connection retries left, this method will either try to reestablish the TCP
* connection or report an I/O error to all tasks which have queued envelopes for this connection. In the latter
* case all queued envelopes will be dropped and all included buffers will be freed.
* <p>
* This method should only be called by the {@link OutgoingConnectionThread} object.
*
* @param ioe
* thrown if an error occurs while reseting the underlying TCP connection
*/
public void reportConnectionProblem(IOException ioe) {
// First, write exception to log
final long currentTime = System.currentTimeMillis();
if (currentTime - this.timstampOfLastRetry >= RETRYINTERVAL) {
LOG.error("Cannot connect to " + this.remoteReceiver + ", " + this.retriesLeft + " retries left");
}
synchronized (this.queuedEnvelopes) {
if (this.selectionKey != null) {
final SocketChannel socketChannel = (SocketChannel) this.selectionKey.channel();
if (socketChannel != null) {
try {
socketChannel.close();
} catch (IOException e) {
LOG.debug("Error while trying to close the socket channel to " + this.remoteReceiver);
}
}
this.selectionKey.cancel();
this.selectionKey = null;
this.isConnected = false;
this.isSubscribedToWriteEvent = false;
}
if (hasRetriesLeft(currentTime)) {
this.connectionThread.triggerConnect(this);
this.isConnected = true;
this.isSubscribedToWriteEvent = true;
return;
}
// Error is fatal
LOG.error(ioe);
// Notify source of current envelope and release buffer
if (this.currentEnvelope != null) {
if (this.currentEnvelope.getBuffer() != null) {
this.currentEnvelope.getBuffer().recycleBuffer();
this.currentEnvelope = null;
}
}
// Notify all other tasks which are waiting for data to be transmitted
final Iterator<Envelope> iter = this.queuedEnvelopes.iterator();
while (iter.hasNext()) {
final Envelope envelope = iter.next();
iter.remove();
// Recycle the buffer inside the envelope
if (envelope.getBuffer() != null) {
envelope.getBuffer().recycleBuffer();
}
}
this.queuedEnvelopes.clear();
}
}
/**
* Reports an I/O error which occurred while writing data to the TCP connection. As a result of the I/O error the
* connection is closed and the interest keys are canceled. Moreover, the task which queued the currently
* transmitted transfer envelope is notified about the error and the current envelope is dropped. If the current
* envelope contains a buffer, the buffer is freed.
* <p>
* This method should only be called by the {@link OutgoingConnectionThread} object.
*
* @param ioe
* thrown if an error occurs while reseting the connection
*/
public void reportTransmissionProblem(IOException ioe) {
final SocketChannel socketChannel = (SocketChannel) this.selectionKey.channel();
// First, write exception to log
if (this.currentEnvelope != null) {
LOG.error("The connection between " + socketChannel.socket().getLocalAddress() + " and "
+ socketChannel.socket().getRemoteSocketAddress()
+ " experienced an IOException for transfer envelope " + this.currentEnvelope.getSequenceNumber());
} else {
LOG.error("The connection between " + socketChannel.socket().getLocalAddress() + " and "
+ socketChannel.socket().getRemoteSocketAddress() + " experienced an IOException");
}
// Close the connection and cancel the interest key
synchronized (this.queuedEnvelopes) {
try {
LOG.debug("Closing connection to " + socketChannel.socket().getRemoteSocketAddress());
socketChannel.close();
} catch (IOException e) {
LOG.debug("An error occurred while responding to an IOException");
LOG.debug(e);
}
this.selectionKey.cancel();
// Error is fatal
LOG.error(ioe);
// Trigger new connection if there are more envelopes to be transmitted
if (this.queuedEnvelopes.isEmpty()) {
this.isConnected = false;
this.isSubscribedToWriteEvent = false;
} else {
this.connectionThread.triggerConnect(this);
this.isConnected = true;
this.isSubscribedToWriteEvent = true;
}
// We must assume the current envelope is corrupted so we notify the task which created it.
if (this.currentEnvelope != null) {
if (this.currentEnvelope.getBuffer() != null) {
this.currentEnvelope.getBuffer().recycleBuffer();
this.currentEnvelope = null;
}
}
}
}
/**
* Checks whether further retries are left for establishing the underlying TCP connection.
*
* @param currentTime
* the current system time in milliseconds since January 1st, 1970
* @return <code>true</code> if there are retries left, <code>false</code> otherwise
*/
private boolean hasRetriesLeft(long currentTime) {
if (currentTime - this.timstampOfLastRetry >= RETRYINTERVAL) {
this.retriesLeft--;
this.timstampOfLastRetry = currentTime;
if (this.retriesLeft == 0) {
return false;
}
}
return true;
}
/**
* Writes the content of the current {@link eu.stratosphere.runtime.io.network.envelope.Envelope} object to the underlying TCP connection.
* <p>
* This method should only be called by the {@link OutgoingConnectionThread} object.
*
* @return <code>true</code> if there is more data from this/other queued envelopes to be written to this channel
* @throws IOException
* thrown if an error occurs while writing the data to the channel
*/
public boolean write() throws IOException {
final WritableByteChannel writableByteChannel = (WritableByteChannel) this.selectionKey.channel();
if (this.currentEnvelope == null) {
synchronized (this.queuedEnvelopes) {
if (this.queuedEnvelopes.isEmpty()) {
return false;
} else {
this.currentEnvelope = this.queuedEnvelopes.peek();
this.writer.setEnvelopeForWriting(this.currentEnvelope);
}
}
}
if (!this.writer.writeNextChunk(writableByteChannel)) {
// Make sure we recycle the attached memory or file buffers correctly
if (this.currentEnvelope.getBuffer() != null) {
this.currentEnvelope.getBuffer().recycleBuffer();
}
synchronized (this.queuedEnvelopes) {
this.queuedEnvelopes.poll();
this.currentEnvelope = null;
}
}
return true;
}
/**
* Requests to close the underlying TCP connection. The request is ignored if at least one {@link eu.stratosphere.runtime.io.network.envelope.Envelope}
* is queued.
* <p>
* This method should only be called by the {@link OutgoingConnectionThread} object.
*
* @throws IOException
* thrown if an error occurs while closing the TCP connection
*/
public void requestClose() throws IOException {
synchronized (this.queuedEnvelopes) {
if (this.queuedEnvelopes.isEmpty()) {
if (this.isSubscribedToWriteEvent) {
this.connectionThread.unsubscribeFromWriteEvent(this.selectionKey);
this.isSubscribedToWriteEvent = false;
}
}
}
}
/**
* Closes the underlying TCP connection if no more {@link eu.stratosphere.runtime.io.network.envelope.Envelope} objects are in the transmission queue.
* <p>
* This method should only be called by the {@link OutgoingConnectionThread} object.
*
* @throws IOException
*/
public void closeConnection() throws IOException {
synchronized (this.queuedEnvelopes) {
if (!this.queuedEnvelopes.isEmpty()) {
return;
}
if (this.selectionKey != null) {
final SocketChannel socketChannel = (SocketChannel) this.selectionKey.channel();
socketChannel.close();
this.selectionKey.cancel();
this.selectionKey = null;
}
this.isConnected = false;
this.isSubscribedToWriteEvent = false;
}
}
/**
* Returns the number of queued {@link eu.stratosphere.runtime.io.network.envelope.Envelope} objects with the given source channel ID.
*
* @param sourceChannelID
* the source channel ID to count the queued envelopes for
* @return the number of queued transfer envelopes with the given source channel ID
*/
public int getNumberOfQueuedEnvelopesFromChannel(final ChannelID sourceChannelID) {
synchronized (this.queuedEnvelopes) {
int number = 0;
final Iterator<Envelope> it = this.queuedEnvelopes.iterator();
while (it.hasNext()) {
final Envelope te = it.next();
if (sourceChannelID.equals(te.getSource())) {
number++;
}
}
return number;
}
}
/**
* Removes all queued {@link eu.stratosphere.runtime.io.network.envelope.Envelope} objects from the transmission which match the given source channel
* ID.
*
* @param sourceChannelID
* the source channel ID of the transfered transfer envelopes to be dropped
*/
public void dropAllQueuedEnvelopesFromChannel(final ChannelID sourceChannelID) {
synchronized (this.queuedEnvelopes) {
final Iterator<Envelope> it = this.queuedEnvelopes.iterator();
while (it.hasNext()) {
final Envelope te = it.next();
if (sourceChannelID.equals(te.getSource())) {
it.remove();
if (te.getBuffer() != null) {
te.getBuffer().recycleBuffer();
}
}
}
}
}
/**
* Checks whether this outgoing connection object manages an active connection or can be removed by the
* {@link eu.stratosphere.nephele.taskmanager.io.bytebuffered.ChannelManager} object.
* <p>
* This method should only be called by the byte buffered channel manager.
*
* @return <code>true</code> if this object is no longer manages an active connection and can be removed,
* <code>false</code> otherwise.
*/
public boolean canBeRemoved() {
synchronized (this.queuedEnvelopes) {
if (this.isConnected) {
return false;
}
if (this.currentEnvelope != null) {
return false;
}
return this.queuedEnvelopes.isEmpty();
}
}
/**
* Sets the selection key representing the interest set of the underlying TCP NIO connection.
*
* @param selectionKey
* the selection of the underlying TCP connection
*/
public void setSelectionKey(SelectionKey selectionKey) {
this.selectionKey = selectionKey;
}
/**
* Returns the number of currently queued envelopes which contain a write buffer.
*
* @return the number of currently queued envelopes which contain a write buffer
*/
public int getNumberOfQueuedWriteBuffers() {
int retVal = 0;
synchronized (this.queuedEnvelopes) {
final Iterator<Envelope> it = this.queuedEnvelopes.iterator();
while (it.hasNext()) {
final Envelope envelope = it.next();
if (envelope.getBuffer() != null) {
++retVal;
}
}
}
return retVal;
}
}
/***********************************************************************************************************************
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network.tcp;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.SelectionKey;
import java.nio.channels.Selector;
import java.nio.channels.SocketChannel;
import java.util.ArrayDeque;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.Queue;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import eu.stratosphere.util.StringUtils;
public class OutgoingConnectionThread extends Thread {
/**
* The minimum time a TCP connection must be idle it is closed.
*/
private static final long MIN_IDLE_TIME_BEFORE_CLOSE = 80000L; // 80 seconds
private static final Log LOG = LogFactory.getLog(OutgoingConnectionThread.class);
private final Selector selector;
private final Queue<OutgoingConnection> pendingConnectionRequests = new ArrayDeque<OutgoingConnection>();
private final Queue<SelectionKey> pendingWriteEventSubscribeRequests = new ArrayDeque<SelectionKey>();
private final Map<OutgoingConnection, Long> connectionsToClose = new HashMap<OutgoingConnection, Long>();
public OutgoingConnectionThread() throws IOException {
super("Outgoing Connection Thread");
this.selector = Selector.open();
}
@Override
public void run() {
try {
while (!isInterrupted()) {
synchronized (this.pendingConnectionRequests) {
if (!this.pendingConnectionRequests.isEmpty()) {
final OutgoingConnection outgoingConnection = this.pendingConnectionRequests.poll();
try {
final SocketChannel socketChannel = SocketChannel.open();
socketChannel.configureBlocking(false);
final SelectionKey key = socketChannel.register(this.selector, SelectionKey.OP_CONNECT);
socketChannel.connect(outgoingConnection.getConnectionAddress());
key.attach(outgoingConnection);
} catch (final IOException ioe) {
// IOException is reported by separate thread to avoid deadlocks
final Runnable reporterThread = new Runnable() {
@Override
public void run() {
outgoingConnection.reportConnectionProblem(ioe);
}
};
new Thread(reporterThread).start();
}
}
}
synchronized (this.pendingWriteEventSubscribeRequests) {
if (!this.pendingWriteEventSubscribeRequests.isEmpty()) {
final SelectionKey oldSelectionKey = this.pendingWriteEventSubscribeRequests.poll();
final OutgoingConnection outgoingConnection = (OutgoingConnection) oldSelectionKey.attachment();
final SocketChannel socketChannel = (SocketChannel) oldSelectionKey.channel();
try {
final SelectionKey newSelectionKey = socketChannel.register(this.selector, SelectionKey.OP_READ
| SelectionKey.OP_WRITE);
newSelectionKey.attach(outgoingConnection);
outgoingConnection.setSelectionKey(newSelectionKey);
} catch (final IOException ioe) {
// IOException is reported by separate thread to avoid deadlocks
final Runnable reporterThread = new Runnable() {
@Override
public void run() {
outgoingConnection.reportTransmissionProblem(ioe);
}
};
new Thread(reporterThread).start();
}
}
}
synchronized (this.connectionsToClose) {
final Iterator<Map.Entry<OutgoingConnection, Long>> closeIt = this.connectionsToClose.entrySet()
.iterator();
final long now = System.currentTimeMillis();
while (closeIt.hasNext()) {
final Map.Entry<OutgoingConnection, Long> entry = closeIt.next();
if ((entry.getValue().longValue() + MIN_IDLE_TIME_BEFORE_CLOSE) < now) {
final OutgoingConnection outgoingConnection = entry.getKey();
closeIt.remove();
// Create new thread to close connection to avoid deadlocks
final Runnable closeThread = new Runnable() {
@Override
public void run() {
try {
outgoingConnection.closeConnection();
} catch (IOException ioe) {
outgoingConnection.reportTransmissionProblem(ioe);
}
}
};
new Thread(closeThread).start();
}
}
}
try {
this.selector.select(10);
} catch (IOException e) {
LOG.error(e);
}
final Iterator<SelectionKey> iter = this.selector.selectedKeys().iterator();
while (iter.hasNext()) {
final SelectionKey key = iter.next();
iter.remove();
if (key.isValid()) {
if (key.isConnectable()) {
doConnect(key);
} else {
if (key.isReadable()) {
doRead(key);
// A read will always result in an exception, so the write key will not be valid anymore
continue;
}
if (key.isWritable()) {
doWrite(key);
}
}
} else {
LOG.error("Received invalid key: " + key);
}
}
}
// Finally, try to close the selector
try {
this.selector.close();
} catch (IOException ioe) {
LOG.debug(StringUtils.stringifyException(ioe));
}
}
catch (Throwable t) {
// this is a disaster, this task manager cannot go on!
LOG.fatal("Outgoing connection thread died with an exception: " + t.getMessage(), t);
System.exit(1);
}
}
private void doConnect(SelectionKey key) {
final OutgoingConnection outgoingConnection = (OutgoingConnection) key.attachment();
final SocketChannel socketChannel = (SocketChannel) key.channel();
try {
while (!socketChannel.finishConnect()) {
try {
Thread.sleep(100);
} catch (InterruptedException e1) {
LOG.error(e1);
}
}
final SelectionKey channelKey = socketChannel.register(selector, SelectionKey.OP_WRITE
| SelectionKey.OP_READ);
outgoingConnection.setSelectionKey(channelKey);
channelKey.attach(outgoingConnection);
} catch (IOException ioe) {
outgoingConnection.reportConnectionProblem(ioe);
}
}
private void doWrite(SelectionKey key) {
final OutgoingConnection outgoingConnection = (OutgoingConnection) key.attachment();
try {
if (!outgoingConnection.write()) {
// Try to close the connection
outgoingConnection.requestClose();
}
} catch (IOException ioe) {
outgoingConnection.reportTransmissionProblem(ioe);
}
}
private void doRead(SelectionKey key) {
final SocketChannel socketChannel = (SocketChannel) key.channel();
final OutgoingConnection outgoingConnection = (OutgoingConnection) key.attachment();
final ByteBuffer buf = ByteBuffer.allocate(8);
try {
if (socketChannel.read(buf) == -1) {
outgoingConnection.reportTransmissionProblem(new IOException(
"Read unexpected EOF from channel"));
} else {
LOG.error("Outgoing connection read real data from channel");
}
} catch (IOException ioe) {
outgoingConnection.reportTransmissionProblem(ioe);
}
}
public void triggerConnect(OutgoingConnection outgoingConnection) {
synchronized (this.pendingConnectionRequests) {
this.pendingConnectionRequests.add(outgoingConnection);
}
}
public void unsubscribeFromWriteEvent(SelectionKey selectionKey) throws IOException {
final SocketChannel socketChannel = (SocketChannel) selectionKey.channel();
final OutgoingConnection outgoingConnection = (OutgoingConnection) selectionKey.attachment();
final SelectionKey newSelectionKey = socketChannel.register(this.selector, SelectionKey.OP_READ);
newSelectionKey.attach(outgoingConnection);
outgoingConnection.setSelectionKey(newSelectionKey);
synchronized (this.connectionsToClose) {
this.connectionsToClose.put(outgoingConnection, Long.valueOf(System.currentTimeMillis()));
}
}
public void subscribeToWriteEvent(SelectionKey selectionKey) {
synchronized (this.pendingWriteEventSubscribeRequests) {
this.pendingWriteEventSubscribeRequests.add(selectionKey);
}
synchronized (this.connectionsToClose) {
this.connectionsToClose.remove((OutgoingConnection) selectionKey.attachment());
}
}
}
...@@ -15,9 +15,6 @@ package eu.stratosphere.runtime.io.serialization; ...@@ -15,9 +15,6 @@ package eu.stratosphere.runtime.io.serialization;
import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.IOReadableWritable;
import eu.stratosphere.core.memory.MemorySegment; import eu.stratosphere.core.memory.MemorySegment;
import eu.stratosphere.runtime.io.serialization.DataInputDeserializer;
import eu.stratosphere.runtime.io.serialization.DataOutputSerializer;
import eu.stratosphere.runtime.io.serialization.RecordDeserializer;
import java.io.DataInput; import java.io.DataInput;
import java.io.EOFException; import java.io.EOFException;
...@@ -62,6 +59,7 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im ...@@ -62,6 +59,7 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
// check if we can get a full length; // check if we can get a full length;
if (nonSpanningRemaining >= 4) { if (nonSpanningRemaining >= 4) {
int len = this.nonSpanningWrapper.readInt(); int len = this.nonSpanningWrapper.readInt();
if (len <= nonSpanningRemaining - 4) { if (len <= nonSpanningRemaining - 4) {
// we can get a full record from here // we can get a full record from here
target.read(this.nonSpanningWrapper); target.read(this.nonSpanningWrapper);
...@@ -156,8 +154,9 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im ...@@ -156,8 +154,9 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
@Override @Override
public final void readFully(byte[] b, int off, int len) throws IOException { public final void readFully(byte[] b, int off, int len) throws IOException {
if (off < 0 || len < 0 || off + len > b.length) if (off < 0 || len < 0 || off + len > b.length) {
throw new IndexOutOfBoundsException(); throw new IndexOutOfBoundsException();
}
this.segment.get(this.position, b, off, len); this.segment.get(this.position, b, off, len);
this.position += len; this.position += len;
...@@ -230,14 +229,16 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im ...@@ -230,14 +229,16 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
try { try {
int b; int b;
while ((b = readUnsignedByte()) != '\n') { while ((b = readUnsignedByte()) != '\n') {
if (b != '\r') if (b != '\r') {
bld.append((char) b); bld.append((char) b);
} }
} }
}
catch (EOFException eofex) {} catch (EOFException eofex) {}
if (bld.length() == 0) if (bld.length() == 0) {
return null; return null;
}
// trim a trailing carriage return // trim a trailing carriage return
int len = bld.length(); int len = bld.length();
...@@ -275,8 +276,9 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im ...@@ -275,8 +276,9 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
while (count < utflen) { while (count < utflen) {
c = (int) bytearr[count] & 0xff; c = (int) bytearr[count] & 0xff;
if (c > 127) if (c > 127) {
break; break;
}
count++; count++;
chararr[chararr_count++] = (char) c; chararr[chararr_count++] = (char) c;
} }
...@@ -298,21 +300,25 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im ...@@ -298,21 +300,25 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
case 12: case 12:
case 13: case 13:
count += 2; count += 2;
if (count > utflen) if (count > utflen) {
throw new UTFDataFormatException("malformed input: partial character at end"); throw new UTFDataFormatException("malformed input: partial character at end");
}
char2 = (int) bytearr[count - 1]; char2 = (int) bytearr[count - 1];
if ((char2 & 0xC0) != 0x80) if ((char2 & 0xC0) != 0x80) {
throw new UTFDataFormatException("malformed input around byte " + count); throw new UTFDataFormatException("malformed input around byte " + count);
}
chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F)); chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
break; break;
case 14: case 14:
count += 3; count += 3;
if (count > utflen) if (count > utflen) {
throw new UTFDataFormatException("malformed input: partial character at end"); throw new UTFDataFormatException("malformed input: partial character at end");
}
char2 = (int) bytearr[count - 2]; char2 = (int) bytearr[count - 2];
char3 = (int) bytearr[count - 1]; char3 = (int) bytearr[count - 1];
if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
throw new UTFDataFormatException("malformed input around byte " + (count - 1)); throw new UTFDataFormatException("malformed input around byte " + (count - 1));
}
chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0)); chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
break; break;
default: default:
...@@ -325,8 +331,9 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im ...@@ -325,8 +331,9 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
@Override @Override
public final int skipBytes(int n) throws IOException { public final int skipBytes(int n) throws IOException {
if (n < 0) if (n < 0) {
throw new IllegalArgumentException(); throw new IllegalArgumentException();
}
int toSkip = Math.min(n, remaining()); int toSkip = Math.min(n, remaining());
this.position += toSkip; this.position += toSkip;
...@@ -390,6 +397,7 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im ...@@ -390,6 +397,7 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
return; return;
} else { } else {
this.recordLength = this.lengthBuffer.getInt(0); this.recordLength = this.lengthBuffer.getInt(0);
this.lengthBuffer.clear(); this.lengthBuffer.clear();
segmentPosition = toPut; segmentPosition = toPut;
} }
......
...@@ -63,11 +63,13 @@ public class DataInputDeserializer implements DataInput { ...@@ -63,11 +63,13 @@ public class DataInputDeserializer implements DataInput {
} }
public void setBuffer(byte[] buffer, int start, int len) { public void setBuffer(byte[] buffer, int start, int len) {
if (buffer == null) if (buffer == null) {
throw new NullPointerException(); throw new NullPointerException();
}
if (start < 0 || len < 0 || start + len >= buffer.length) if (start < 0 || len < 0 || start + len >= buffer.length) {
throw new IllegalArgumentException(); throw new IllegalArgumentException();
}
this.buffer = buffer; this.buffer = buffer;
this.position = start; this.position = start;
...@@ -215,8 +217,9 @@ public class DataInputDeserializer implements DataInput { ...@@ -215,8 +217,9 @@ public class DataInputDeserializer implements DataInput {
while (count < utflen) { while (count < utflen) {
c = (int) bytearr[count] & 0xff; c = (int) bytearr[count] & 0xff;
if (c > 127) if (c > 127) {
break; break;
}
count++; count++;
chararr[chararr_count++] = (char) c; chararr[chararr_count++] = (char) c;
} }
...@@ -240,22 +243,26 @@ public class DataInputDeserializer implements DataInput { ...@@ -240,22 +243,26 @@ public class DataInputDeserializer implements DataInput {
case 13: case 13:
/* 110x xxxx 10xx xxxx */ /* 110x xxxx 10xx xxxx */
count += 2; count += 2;
if (count > utflen) if (count > utflen) {
throw new UTFDataFormatException("malformed input: partial character at end"); throw new UTFDataFormatException("malformed input: partial character at end");
}
char2 = (int) bytearr[count - 1]; char2 = (int) bytearr[count - 1];
if ((char2 & 0xC0) != 0x80) if ((char2 & 0xC0) != 0x80) {
throw new UTFDataFormatException("malformed input around byte " + count); throw new UTFDataFormatException("malformed input around byte " + count);
}
chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F)); chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
break; break;
case 14: case 14:
/* 1110 xxxx 10xx xxxx 10xx xxxx */ /* 1110 xxxx 10xx xxxx 10xx xxxx */
count += 3; count += 3;
if (count > utflen) if (count > utflen) {
throw new UTFDataFormatException("malformed input: partial character at end"); throw new UTFDataFormatException("malformed input: partial character at end");
}
char2 = (int) bytearr[count - 2]; char2 = (int) bytearr[count - 2];
char3 = (int) bytearr[count - 1]; char3 = (int) bytearr[count - 1];
if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
throw new UTFDataFormatException("malformed input around byte " + (count - 1)); throw new UTFDataFormatException("malformed input around byte " + (count - 1));
}
chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0)); chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
break; break;
default: default:
......
...@@ -196,9 +196,9 @@ public class DataOutputSerializer implements DataOutput { ...@@ -196,9 +196,9 @@ public class DataOutputSerializer implements DataOutput {
} }
} }
if (utflen > 65535) if (utflen > 65535) {
throw new UTFDataFormatException("Encoded string is too long: " + utflen); throw new UTFDataFormatException("Encoded string is too long: " + utflen);
}
else if (this.position > this.buffer.length - utflen - 2) { else if (this.position > this.buffer.length - utflen - 2) {
resize(utflen + 2); resize(utflen + 2);
} }
...@@ -212,8 +212,9 @@ public class DataOutputSerializer implements DataOutput { ...@@ -212,8 +212,9 @@ public class DataOutputSerializer implements DataOutput {
int i = 0; int i = 0;
for (i = 0; i < strlen; i++) { for (i = 0; i < strlen; i++) {
c = str.charAt(i); c = str.charAt(i);
if (!((c >= 0x0001) && (c <= 0x007F))) if (!((c >= 0x0001) && (c <= 0x007F))) {
break; break;
}
bytearr[count++] = (byte) c; bytearr[count++] = (byte) c;
} }
......
...@@ -68,6 +68,7 @@ public class SpanningRecordSerializer<T extends IOReadableWritable> implements R ...@@ -68,6 +68,7 @@ public class SpanningRecordSerializer<T extends IOReadableWritable> implements R
// write data and length // write data and length
record.write(this.serializationBuffer); record.write(this.serializationBuffer);
this.lengthBuffer.putInt(0, this.serializationBuffer.length()); this.lengthBuffer.putInt(0, this.serializationBuffer.length());
this.dataBuffer = this.serializationBuffer.wrapAsByteBuffer(); this.dataBuffer = this.serializationBuffer.wrapAsByteBuffer();
...@@ -103,8 +104,9 @@ public class SpanningRecordSerializer<T extends IOReadableWritable> implements R ...@@ -103,8 +104,9 @@ public class SpanningRecordSerializer<T extends IOReadableWritable> implements R
* @param source the {@link ByteBuffer} to copy data from * @param source the {@link ByteBuffer} to copy data from
*/ */
private void copyToTargetBufferFrom(ByteBuffer source) { private void copyToTargetBufferFrom(ByteBuffer source) {
if (this.targetBuffer == null) if (this.targetBuffer == null) {
return; return;
}
int needed = source.remaining(); int needed = source.remaining();
int available = this.limit - this.position; int available = this.limit - this.position;
...@@ -127,8 +129,9 @@ public class SpanningRecordSerializer<T extends IOReadableWritable> implements R ...@@ -127,8 +129,9 @@ public class SpanningRecordSerializer<T extends IOReadableWritable> implements R
@Override @Override
public Buffer getCurrentBuffer() { public Buffer getCurrentBuffer() {
if (targetBuffer == null) if (targetBuffer == null) {
return null; return null;
}
this.targetBuffer.limitSize(this.position); this.targetBuffer.limitSize(this.position);
return this.targetBuffer; return this.targetBuffer;
......
...@@ -70,7 +70,7 @@ public class TestBufferProvider implements BufferProvider { ...@@ -70,7 +70,7 @@ public class TestBufferProvider implements BufferProvider {
} }
@Override @Override
public boolean registerBufferAvailabilityListener(BufferAvailabilityListener bufferAvailabilityListener) { public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener bufferAvailabilityListener) {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
} }
...@@ -33,7 +33,6 @@ import eu.stratosphere.api.java.typeutils.runtime.record.RecordSerializerFactory ...@@ -33,7 +33,6 @@ import eu.stratosphere.api.java.typeutils.runtime.record.RecordSerializerFactory
import eu.stratosphere.core.memory.DataInputView; import eu.stratosphere.core.memory.DataInputView;
import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.core.memory.DataOutputView;
import eu.stratosphere.core.memory.MemorySegment; import eu.stratosphere.core.memory.MemorySegment;
import eu.stratosphere.nephele.io.ChannelSelector;
import eu.stratosphere.pact.runtime.plugable.SerializationDelegate; import eu.stratosphere.pact.runtime.plugable.SerializationDelegate;
import eu.stratosphere.pact.runtime.shipping.OutputEmitter; import eu.stratosphere.pact.runtime.shipping.OutputEmitter;
import eu.stratosphere.pact.runtime.shipping.ShipStrategyType; import eu.stratosphere.pact.runtime.shipping.ShipStrategyType;
......
...@@ -28,8 +28,6 @@ import org.junit.Test; ...@@ -28,8 +28,6 @@ import org.junit.Test;
import eu.stratosphere.api.common.distributions.DataDistribution; import eu.stratosphere.api.common.distributions.DataDistribution;
import eu.stratosphere.api.common.distributions.UniformIntegerDistribution; import eu.stratosphere.api.common.distributions.UniformIntegerDistribution;
import eu.stratosphere.runtime.io.api.ChannelSelector; import eu.stratosphere.runtime.io.api.ChannelSelector;
import eu.stratosphere.pact.runtime.plugable.pactrecord.RecordComparator;
import eu.stratosphere.nephele.io.ChannelSelector;
import eu.stratosphere.api.java.typeutils.runtime.record.RecordComparator; import eu.stratosphere.api.java.typeutils.runtime.record.RecordComparator;
import eu.stratosphere.pact.runtime.shipping.RecordOutputEmitter; import eu.stratosphere.pact.runtime.shipping.RecordOutputEmitter;
import eu.stratosphere.pact.runtime.shipping.ShipStrategyType; import eu.stratosphere.pact.runtime.shipping.ShipStrategyType;
......
...@@ -44,7 +44,9 @@ import eu.stratosphere.util.MutableObjectIterator; ...@@ -44,7 +44,9 @@ import eu.stratosphere.util.MutableObjectIterator;
import java.io.IOException; import java.io.IOException;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.FutureTask;
public class MockEnvironment implements Environment, BufferProvider, LocalBufferPoolOwner { public class MockEnvironment implements Environment, BufferProvider, LocalBufferPoolOwner {
...@@ -124,8 +126,8 @@ public class MockEnvironment implements Environment, BufferProvider, LocalBuffer ...@@ -124,8 +126,8 @@ public class MockEnvironment implements Environment, BufferProvider, LocalBuffer
} }
@Override @Override
public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) { public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
return false; return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
} }
@Override @Override
...@@ -338,4 +340,9 @@ public class MockEnvironment implements Environment, BufferProvider, LocalBuffer ...@@ -338,4 +340,9 @@ public class MockEnvironment implements Environment, BufferProvider, LocalBuffer
public BufferProvider getOutputBufferProvider() { public BufferProvider getOutputBufferProvider() {
return this; return this;
} }
@Override
public Map<String, FutureTask<Path>> getCopyTask() {
return null;
}
} }
/***********************************************************************************************************************
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network.envelope;
import eu.stratosphere.core.memory.MemorySegment;
import eu.stratosphere.nephele.event.task.AbstractEvent;
import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.nephele.util.DiscardingRecycler;
import eu.stratosphere.nephele.util.TestBufferProvider;
import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProviderBroker;
import eu.stratosphere.runtime.io.BufferRecycler;
import eu.stratosphere.runtime.io.channels.ChannelID;
import org.junit.Assert;
import org.junit.Test;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.File;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.WritableByteChannel;
import java.util.Arrays;
import java.util.List;
import java.util.Random;
public class EnvelopeReaderWriterTest {
private final long RANDOM_SEED = 520346508276087l;
private static final int BUFFER_SIZE = 32768;
private static final byte BUFFER_CONTENT = 13;
private final int[] BUFFER_SIZES = { 0, 2, BUFFER_SIZE, 3782, 88, 0, 23};
private final AbstractEvent[][] EVENT_LISTS = {
{},
{},
{},
{ new TestEvent1(34872527) },
{ new TestEvent1(8749653), new TestEvent1(365345) },
{ new TestEvent2(34563456), new TestEvent1(598432), new TestEvent2(976293845) },
{}
};
@Test
public void testWriteAndRead() {
Assert.assertTrue("Test broken.", BUFFER_SIZES.length == EVENT_LISTS.length);
File testFile = null;
RandomAccessFile raf = null;
try {
testFile = File.createTempFile("envelopes", ".tmp");
raf = new RandomAccessFile(testFile, "rw");
// write
FileChannel c = raf.getChannel();
writeEnvelopes(c);
// read
c.position(0);
readEnvelopes(c, -1.0f);
c.close();
}
catch (Exception e) {
System.err.println(e.getMessage());
e.printStackTrace();
Assert.fail(e.getMessage());
}
finally {
if (raf != null)
try { raf.close(); } catch (Throwable t) {}
if (testFile != null)
testFile.delete();
}
}
@Test
public void testWriteAndReadChunked() {
Assert.assertTrue("Test broken.", BUFFER_SIZES.length == EVENT_LISTS.length);
File testFile = null;
RandomAccessFile raf = null;
try {
testFile = File.createTempFile("envelopes", ".tmp");
raf = new RandomAccessFile(testFile, "rw");
// write
FileChannel c = raf.getChannel();
writeEnvelopes(new ChunkedWriteableChannel(c));
// read
c.position(0);
readEnvelopes(new ChunkedReadableChannel(c), 0.75f);
c.close();
}
catch (Exception e) {
System.err.println(e.getMessage());
e.printStackTrace();
Assert.fail(e.getMessage());
}
finally {
if (raf != null)
try { raf.close(); } catch (Throwable t) {}
if (testFile != null)
testFile.delete();
}
}
private void writeEnvelopes(WritableByteChannel channel) throws IOException {
final BufferRecycler recycler = new DiscardingRecycler();
final Random rand = new Random(RANDOM_SEED);
final EnvelopeWriter serializer = new EnvelopeWriter();
final int NUM_ENVS = BUFFER_SIZES.length;
for (int i = 0; i < NUM_ENVS; i++) {
int seqNum = Math.abs(rand.nextInt());
JobID jid = new JobID(rand.nextLong(), rand.nextLong());
ChannelID sid = new ChannelID(rand.nextLong(), rand.nextLong());
Envelope env = new Envelope(seqNum, jid, sid);
if (EVENT_LISTS[i].length > 0) {
env.serializeEventList(Arrays.asList(EVENT_LISTS[i]));
}
int bufferSize = BUFFER_SIZES[i];
if (bufferSize > 0) {
MemorySegment ms = new MemorySegment(new byte[BUFFER_SIZE]);
for (int x = 0; x < bufferSize; x++) {
ms.put(x, BUFFER_CONTENT);
}
Buffer mb = new Buffer(ms, bufferSize, recycler);
env.setBuffer(mb);
}
serializer.setEnvelopeForWriting(env);
while (serializer.writeNextChunk(channel));
}
}
private void readEnvelopes(ReadableByteChannel channel, float probabilityForNoBufferCurrently) throws IOException {
final Random rand = new Random(RANDOM_SEED);
final EnvelopeReader reader = new EnvelopeReader(new OneForAllBroker(BUFFER_SIZE, probabilityForNoBufferCurrently));
final int NUM_ENVS = BUFFER_SIZES.length;
for (int i = 0; i < NUM_ENVS; i++) {
int expectedSeqNum = Math.abs(rand.nextInt());
JobID expectedJid = new JobID(rand.nextLong(), rand.nextLong());
ChannelID expectedSid = new ChannelID(rand.nextLong(), rand.nextLong());
// read the next envelope
while (reader.readNextChunk(channel) != EnvelopeReader.DeserializationState.COMPLETE);
Envelope env = reader.getFullyDeserializedTransferEnvelope();
// check the basic fields from the header
Assert.assertEquals(expectedSeqNum, env.getSequenceNumber());
Assert.assertEquals(expectedJid, env.getJobID());
Assert.assertEquals(expectedSid, env.getSource());
// check the events
List<? extends AbstractEvent> events = env.deserializeEvents();
Assert.assertEquals(EVENT_LISTS[i].length, events.size());
for (int n = 0; n < EVENT_LISTS[i].length; n++) {
AbstractEvent expectedEvent = EVENT_LISTS[i][n];
AbstractEvent actualEvent = events.get(n);
Assert.assertEquals(expectedEvent.getClass(), actualEvent.getClass());
Assert.assertEquals(expectedEvent, actualEvent);
}
// check the buffer
Buffer buf = env.getBuffer();
if (buf == null) {
Assert.assertTrue(BUFFER_SIZES[i] == 0);
} else {
Assert.assertEquals(BUFFER_SIZES[i], buf.size());
for (int k = 0; k < BUFFER_SIZES[i]; k++) {
Assert.assertEquals(BUFFER_CONTENT, buf.getMemorySegment().get(k));
}
}
reader.reset();
}
}
public static final class TestEvent1 extends AbstractEvent {
private long id;
public TestEvent1() {}
public TestEvent1(long id) {
this.id = id;
}
@Override
public void write(DataOutput out) throws IOException {
out.writeLong(id);
}
@Override
public void read(DataInput in) throws IOException {
id = in.readLong();
}
@Override
public boolean equals(Object obj) {
if (obj.getClass() == TestEvent1.class) {
return ((TestEvent1) obj).id == this.id;
} else {
return false;
}
}
@Override
public int hashCode() {
return ((int) id) ^ ((int) (id >>> 32));
}
@Override
public String toString() {
return "TestEvent1 (" + id + ")";
}
}
public static final class TestEvent2 extends AbstractEvent {
private long id;
public TestEvent2() {}
public TestEvent2(long id) {
this.id = id;
}
@Override
public void write(DataOutput out) throws IOException {
out.writeLong(id);
}
@Override
public void read(DataInput in) throws IOException {
id = in.readLong();
}
@Override
public boolean equals(Object obj) {
if (obj.getClass() == TestEvent2.class) {
return ((TestEvent2) obj).id == this.id;
} else {
return false;
}
}
@Override
public int hashCode() {
return ((int) id) ^ ((int) (id >>> 32));
}
@Override
public String toString() {
return "TestEvent2 (" + id + ")";
}
}
private static final class ChunkedWriteableChannel implements WritableByteChannel {
private final WritableByteChannel delegate;
private final Random rnd;
private ChunkedWriteableChannel(WritableByteChannel delegate) {
this.delegate = delegate;
this.rnd = new Random();
}
@Override
public boolean isOpen() {
return this.delegate.isOpen();
}
@Override
public void close() throws IOException {
this.delegate.close();
}
@Override
public int write(ByteBuffer src) throws IOException {
final int available = src.remaining();
final int oldLimit = src.limit();
int toWrite = rnd.nextInt(available) + 1;
toWrite = Math.min(Math.max(toWrite, 8), available);
src.limit(src.position() + toWrite);
int written = this.delegate.write(src);
src.limit(oldLimit);
return written;
}
}
private static final class ChunkedReadableChannel implements ReadableByteChannel {
private final ReadableByteChannel delegate;
private final Random rnd;
private ChunkedReadableChannel(ReadableByteChannel delegate) {
this.delegate = delegate;
this.rnd = new Random();
}
@Override
public boolean isOpen() {
return this.delegate.isOpen();
}
@Override
public void close() throws IOException {
this.delegate.close();
}
@Override
public int read(ByteBuffer dst) throws IOException {
final int available = dst.remaining();
final int oldLimit = dst.limit();
int toRead = rnd.nextInt(available) + 1;
toRead = Math.min(Math.max(toRead, 8), available);
dst.limit(dst.position() + toRead);
int read = this.delegate.read(dst);
dst.limit(oldLimit);
return read;
}
}
private static final class OneForAllBroker implements BufferProviderBroker {
private final TestBufferProvider provider;
private OneForAllBroker(int sizeOfMemorySegments) {
this.provider = new TestBufferProvider(sizeOfMemorySegments);
}
private OneForAllBroker(int sizeOfMemorySegments, float probabilityForNoBufferCurrently) {
this.provider = new TestBufferProvider(sizeOfMemorySegments, probabilityForNoBufferCurrently);
}
@Override
public BufferProvider getBufferProvider(JobID jobID, ChannelID sourceChannelID) {
return this.provider;
}
}
}
/***********************************************************************************************************************
* Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network.netty;
import eu.stratosphere.core.memory.MemorySegment;
import eu.stratosphere.nephele.event.task.AbstractEvent;
import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.BufferRecycler;
import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider.BufferAvailabilityRegistration;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferProviderBroker;
import eu.stratosphere.runtime.io.network.Envelope;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
import io.netty.buffer.CompositeByteBuf;
import io.netty.channel.embedded.EmbeddedChannel;
import junit.framework.Assert;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Matchers;
import org.mockito.Mock;
import org.mockito.MockitoAnnotations;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.LinkedList;
import java.util.List;
import java.util.Random;
import static org.mockito.Matchers.anyInt;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
public class InboundEnvelopeDecoderTest {
@Mock
private BufferProvider bufferProvider;
@Mock
private BufferProviderBroker bufferProviderBroker;
@Before
public void initMocks() throws IOException {
MockitoAnnotations.initMocks(this);
}
@Test
public void testBufferStaging() throws Exception {
final InboundEnvelopeDecoder decoder = new InboundEnvelopeDecoder(this.bufferProviderBroker);
final EmbeddedChannel ch = new EmbeddedChannel(
new OutboundEnvelopeEncoder(),
decoder);
when(this.bufferProviderBroker.getBufferProvider(anyJobId(), anyChannelId()))
.thenReturn(this.bufferProvider);
// --------------------------------------------------------------------
Envelope[] envelopes = nextEnvelopes(3, true);
ByteBuf buf = encode(ch, envelopes);
when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
.thenReturn(BufferAvailabilityRegistration.REGISTERED);
Buffer buffer = allocBuffer(envelopes[2].getBuffer().size());
when(this.bufferProvider.requestBuffer(anyInt()))
.thenReturn(null, null, buffer, null);
// --------------------------------------------------------------------
// slices: [0] => full envelope, [1] => half envelope, [2] => remaining half + full envelope
ByteBuf[] slices = slice(buf,
OutboundEnvelopeEncoder.HEADER_SIZE + envelopes[0].getBuffer().size(),
OutboundEnvelopeEncoder.HEADER_SIZE + envelopes[1].getBuffer().size() / 2);
// 1. no buffer available, incoming slice contains all data
int refCount = slices[0].refCnt();
decodeAndVerify(ch, slices[0]);
Assert.assertEquals(refCount + 1, slices[0].refCnt());
Assert.assertFalse(ch.config().isAutoRead());
// notify of available buffer (=> bufferAvailable() callback does return a buffer
// of the current network buffer size; the decoder needs to adjust its size to the
// requested size
decoder.bufferAvailable(allocBuffer(envelopes[0].getBuffer().size() * 2));
ch.runPendingTasks();
Assert.assertEquals(refCount - 1, slices[0].refCnt());
Assert.assertTrue(ch.config().isAutoRead());
decodeAndVerify(ch, envelopes[0]);
// 2. no buffer available, incoming slice does NOT contain all data
refCount = slices[1].refCnt();
decodeAndVerify(ch, slices[1]);
Assert.assertEquals(refCount + 1, slices[1].refCnt());
Assert.assertFalse(ch.config().isAutoRead());
decoder.bufferAvailable(allocBuffer());
ch.runPendingTasks();
Assert.assertEquals(refCount - 1, slices[1].refCnt());
Assert.assertTrue(ch.config().isAutoRead());
decodeAndVerify(ch);
// 3. buffer available
refCount = slices[2].refCnt();
decodeAndVerify(ch, slices[2], envelopes[1], envelopes[2]);
Assert.assertEquals(refCount - 1, slices[2].refCnt());
Assert.assertTrue(ch.config().isAutoRead());
Assert.assertEquals(1, buf.refCnt());
buf.release();
}
@Test
public void testBufferStagingStagedBufferException() throws Exception {
final EmbeddedChannel ch = new EmbeddedChannel(
new OutboundEnvelopeEncoder(),
new InboundEnvelopeDecoder(this.bufferProviderBroker));
when(this.bufferProviderBroker.getBufferProvider(anyJobId(), anyChannelId()))
.thenReturn(this.bufferProvider);
// --------------------------------------------------------------------
ByteBuf buf = encode(ch, nextEnvelope(true));
when(this.bufferProvider.requestBuffer(anyInt()))
.thenReturn(null);
when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
.thenReturn(BufferAvailabilityRegistration.REGISTERED);
// --------------------------------------------------------------------
int refCount = buf.refCnt();
decodeAndVerify(ch, buf);
Assert.assertFalse(ch.config().isAutoRead());
Assert.assertEquals(refCount + 1, buf.refCnt());
try {
decodeAndVerify(ch, buf);
Assert.fail("Expected IllegalStateException not thrown");
} catch (IllegalStateException e) {
// expected exception
}
buf.release();
}
@Test
public void testBufferAvailabilityRegistrationBufferAvailable() throws Exception {
final EmbeddedChannel ch = new EmbeddedChannel(
new OutboundEnvelopeEncoder(),
new InboundEnvelopeDecoder(this.bufferProviderBroker));
when(this.bufferProviderBroker.getBufferProvider(anyJobId(), anyChannelId()))
.thenReturn(this.bufferProvider);
// --------------------------------------------------------------------
Envelope[] envelopes = new Envelope[]{nextEnvelope(true), nextEnvelope()};
when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
.thenReturn(BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_AVAILABLE);
when(this.bufferProvider.requestBuffer(anyInt()))
.thenReturn(null)
.thenReturn(allocBuffer(envelopes[0].getBuffer().size()));
// --------------------------------------------------------------------
ByteBuf buf = encode(ch, envelopes);
decodeAndVerify(ch, buf, envelopes);
Assert.assertEquals(0, buf.refCnt());
}
@Test
public void testBufferAvailabilityRegistrationBufferPoolDestroyedSkipBytes() throws Exception {
final EmbeddedChannel ch = new EmbeddedChannel(
new OutboundEnvelopeEncoder(),
new InboundEnvelopeDecoder(this.bufferProviderBroker));
when(this.bufferProviderBroker.getBufferProvider(anyJobId(), anyChannelId()))
.thenReturn(this.bufferProvider);
when(this.bufferProvider.requestBuffer(anyInt()))
.thenReturn(null);
when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
.thenReturn(BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED);
// --------------------------------------------------------------------
Envelope[] envelopes = new Envelope[]{nextEnvelope(true), nextEnvelope(), nextEnvelope()};
Envelope[] expectedEnvelopes = new Envelope[]{envelopes[1], envelopes[2]};
ByteBuf buf = encode(ch, envelopes);
int bufferSize = envelopes[0].getBuffer().size();
// --------------------------------------------------------------------
// 1) skip in current buffer only
// --------------------------------------------------------------------
{
// skip last bytes in current buffer
ByteBuf[] slices = slice(buf, OutboundEnvelopeEncoder.HEADER_SIZE + bufferSize);
int refCount = slices[0].refCnt();
decodeAndVerify(ch, slices[0]);
Assert.assertEquals(refCount - 1, slices[0].refCnt());
refCount = slices[1].refCnt();
decodeAndVerify(ch, slices[1], expectedEnvelopes);
Assert.assertEquals(refCount - 1, slices[1].refCnt());
}
{
// skip bytes in current buffer, leave last 16 bytes from next envelope
ByteBuf[] slices = slice(buf, OutboundEnvelopeEncoder.HEADER_SIZE + bufferSize + 16);
int refCount = slices[0].refCnt();
decodeAndVerify(ch, slices[0]);
Assert.assertEquals(refCount - 1, slices[0].refCnt());
refCount = slices[1].refCnt();
decodeAndVerify(ch, slices[1], expectedEnvelopes);
Assert.assertEquals(refCount - 1, slices[1].refCnt());
}
{
// skip bytes in current buffer, then continue with full envelope from same buffer
ByteBuf[] slices = slice(buf, OutboundEnvelopeEncoder.HEADER_SIZE + bufferSize + OutboundEnvelopeEncoder.HEADER_SIZE);
int refCount = slices[0].refCnt();
decodeAndVerify(ch, slices[0], expectedEnvelopes[0]);
Assert.assertEquals(refCount - 1, slices[0].refCnt());
refCount = slices[1].refCnt();
decodeAndVerify(ch, slices[1], expectedEnvelopes[1]);
Assert.assertEquals(refCount - 1, slices[1].refCnt());
}
// --------------------------------------------------------------------
// 2) skip in current and next buffer
// --------------------------------------------------------------------
{
// skip bytes in current buffer, then continue to skip last 32 bytes in next buffer
ByteBuf[] slices = slice(buf, OutboundEnvelopeEncoder.HEADER_SIZE + bufferSize - 32);
int refCount = slices[0].refCnt();
decodeAndVerify(ch, slices[0]);
Assert.assertEquals(refCount - 1, slices[0].refCnt());
refCount = slices[1].refCnt();
decodeAndVerify(ch, slices[1], expectedEnvelopes);
Assert.assertEquals(refCount - 1, slices[1].refCnt());
}
{
// skip bytes in current buffer, then continue to skip in next two buffers
ByteBuf[] slices = slice(buf, OutboundEnvelopeEncoder.HEADER_SIZE + bufferSize - 32, 16);
int refCount = slices[0].refCnt();
decodeAndVerify(ch, slices[0]);
Assert.assertEquals(refCount - 1, slices[0].refCnt());
refCount = slices[1].refCnt();
decodeAndVerify(ch, slices[1]);
Assert.assertEquals(refCount - 1, slices[1].refCnt());
refCount = slices[2].refCnt();
decodeAndVerify(ch, slices[2], expectedEnvelopes);
Assert.assertEquals(refCount - 1, slices[2].refCnt());
}
// ref count should be 1, because slices shared the ref count
Assert.assertEquals(1, buf.refCnt());
}
@Test
public void testEncodeDecode() throws Exception {
final EmbeddedChannel ch = new EmbeddedChannel(
new OutboundEnvelopeEncoder(), new InboundEnvelopeDecoder(this.bufferProviderBroker));
when(this.bufferProviderBroker.getBufferProvider(anyJobId(), anyChannelId()))
.thenReturn(this.bufferProvider);
when(this.bufferProvider.requestBuffer(anyInt())).thenAnswer(new Answer<Object>() {
@Override
public Object answer(InvocationOnMock invocation) throws Throwable {
// fulfill the buffer request
return allocBuffer((Integer) invocation.getArguments()[0]);
}
});
// --------------------------------------------------------------------
Envelope[] envelopes = new Envelope[]{
nextEnvelope(0),
nextEnvelope(2),
nextEnvelope(32768),
nextEnvelope(3782, new TestEvent1(34872527)),
nextEnvelope(88, new TestEvent1(8749653), new TestEvent1(365345)),
nextEnvelope(0, new TestEvent2(34563456), new TestEvent1(598432), new TestEvent2(976293845)),
nextEnvelope(23)
};
ByteBuf buf = encode(ch, envelopes);
// 1. complete ByteBuf as input
int refCount = buf.retain().refCnt();
decodeAndVerify(ch, buf, envelopes);
Assert.assertEquals(refCount - 1, buf.refCnt());
// 2. random slices
buf.readerIndex(0);
ByteBuf[] slices = randomSlices(buf);
ch.writeInbound(slices);
for (ByteBuf slice : slices) {
Assert.assertEquals(1, slice.refCnt());
}
decodeAndVerify(ch, envelopes);
buf.release();
}
@Test
public void testEncodeDecodeRandomEnvelopes() throws Exception {
final InboundEnvelopeDecoder decoder = new InboundEnvelopeDecoder(this.bufferProviderBroker);
final EmbeddedChannel ch = new EmbeddedChannel(
new OutboundEnvelopeEncoder(), decoder);
when(this.bufferProviderBroker.getBufferProvider(anyJobId(), anyChannelId()))
.thenReturn(this.bufferProvider);
when(this.bufferProvider.requestBuffer(anyInt())).thenAnswer(new Answer<Object>() {
@Override
public Object answer(InvocationOnMock invocation) throws Throwable {
// fulfill the buffer request with the requested size
return allocBuffer((Integer) invocation.getArguments()[0]);
}
});
Random randomAnswerSource = new Random(RANDOM_SEED);
RandomBufferRequestAnswer randomBufferRequestAnswer = new RandomBufferRequestAnswer(randomAnswerSource);
RandomBufferAvailabilityRegistrationAnswer randomBufferAvailabilityRegistrationAnswer =
new RandomBufferAvailabilityRegistrationAnswer(randomAnswerSource, randomBufferRequestAnswer);
when(this.bufferProvider.requestBuffer(anyInt())).thenAnswer(randomBufferRequestAnswer);
when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
.thenAnswer(randomBufferAvailabilityRegistrationAnswer);
// --------------------------------------------------------------------
Envelope[] envelopes = nextRandomEnvelopes(1024);
ByteBuf buf = encode(ch, envelopes);
ByteBuf[] slices = randomSlices(buf);
for (ByteBuf slice : slices) {
int refCount = slice.refCnt();
ch.writeInbound(slice);
// registered BufferAvailabilityListener => call bufferAvailable(buffer)
while (randomBufferAvailabilityRegistrationAnswer.isRegistered()) {
randomBufferAvailabilityRegistrationAnswer.unregister();
Assert.assertFalse(ch.config().isAutoRead());
Assert.assertEquals(refCount + 1, slice.refCnt());
// return a buffer of max size => decoder needs to limit buffer size
decoder.bufferAvailable(allocBuffer(MAX_BUFFER_SIZE));
ch.runPendingTasks();
}
Assert.assertEquals(refCount - 1, slice.refCnt());
Assert.assertTrue(ch.config().isAutoRead());
}
Envelope[] expected = randomBufferAvailabilityRegistrationAnswer.removeSkippedEnvelopes(envelopes);
decodeAndVerify(ch, expected);
Assert.assertEquals(1, buf.refCnt());
buf.release();
}
// ========================================================================
// helpers
// ========================================================================
private final static long RANDOM_SEED = 520346508276087l;
private final static Random random = new Random(RANDOM_SEED);
private final static int[] BUFFER_SIZES = new int[]{8192, 16384, 32768};
private final static int MAX_BUFFER_SIZE = BUFFER_SIZES[2];
private final static int MAX_NUM_EVENTS = 5;
private final static int MAX_SLICE_SIZE = MAX_BUFFER_SIZE / 3;
private final static int MIN_SLICE_SIZE = 1;
private final static BufferRecycler RECYCLER = mock(BufferRecycler.class);
// ------------------------------------------------------------------------
// envelopes
// ------------------------------------------------------------------------
private static Buffer allocBuffer() {
return allocBuffer(MAX_BUFFER_SIZE);
}
private static Buffer allocBuffer(int bufferSize) {
return spy(new Buffer(new MemorySegment(new byte[bufferSize]), bufferSize, RECYCLER));
}
private Envelope nextEnvelope() {
return nextEnvelope(false, false);
}
private Envelope nextEnvelope(boolean withBuffer) {
return nextEnvelope(withBuffer, false);
}
private Envelope nextEnvelope(int bufferSize, AbstractEvent... events) {
Envelope env = new Envelope(random.nextInt(), new JobID(), new ChannelID());
if (bufferSize > 0) {
byte[] data = new byte[bufferSize];
random.nextBytes(data);
env.setBuffer(spy(new Buffer(new MemorySegment(data), bufferSize, RECYCLER)));
}
if (events != null && events.length > 0) {
env.serializeEventList(Arrays.asList(events));
}
return env;
}
private Envelope nextEnvelope(boolean withBuffer, boolean withEvents) {
int bufferSize = 0;
AbstractEvent[] events = null;
if (withBuffer) {
bufferSize = BUFFER_SIZES[random.nextInt(BUFFER_SIZES.length)];
}
if (withEvents) {
events = new AbstractEvent[random.nextInt(MAX_NUM_EVENTS) + 1];
for (int i = 0; i < events.length; i++) {
events[i] = (random.nextBoolean()
? new TestEvent1(random.nextLong())
: new TestEvent2(random.nextLong()));
}
}
return nextEnvelope(bufferSize, events);
}
private Envelope[] nextEnvelopes(int numEnvelopes, boolean withBuffer) {
Envelope[] envelopes = new Envelope[numEnvelopes];
for (int i = 0; i < numEnvelopes; i++) {
envelopes[i] = nextEnvelope(withBuffer, false);
}
return envelopes;
}
private Envelope[] nextRandomEnvelopes(int numEnvelopes) {
Envelope[] envelopes = new Envelope[numEnvelopes];
for (int i = 0; i < numEnvelopes; i++) {
envelopes[i] = nextEnvelope(random.nextBoolean(), random.nextBoolean());
}
return envelopes;
}
// ------------------------------------------------------------------------
// channel encode/decode
// ------------------------------------------------------------------------
private static ByteBuf encode(EmbeddedChannel ch, Envelope... envelopes) {
for (Envelope env : envelopes) {
ch.writeOutbound(env);
if (env.getBuffer() != null) {
verify(env.getBuffer(), times(1)).recycleBuffer();
}
}
CompositeByteBuf encodedEnvelopes = new CompositeByteBuf(ByteBufAllocator.DEFAULT, false, envelopes.length);
ByteBuf buf;
while ((buf = (ByteBuf) ch.readOutbound()) != null) {
encodedEnvelopes.addComponent(buf);
}
return encodedEnvelopes.writerIndex(encodedEnvelopes.capacity());
}
private static void decodeAndVerify(EmbeddedChannel ch, ByteBuf buf, Envelope... expectedEnvelopes) {
ch.writeInbound(buf);
decodeAndVerify(ch, expectedEnvelopes);
}
private static void decodeAndVerify(EmbeddedChannel ch, Envelope... expectedEnvelopes) {
if (expectedEnvelopes == null) {
Assert.assertNull(ch.readInbound());
}
else {
for (Envelope expected : expectedEnvelopes) {
Envelope actual = (Envelope) ch.readInbound();
if (actual == null) {
Assert.fail("No inbound envelope available, but expected one");
}
assertEqualEnvelopes(expected, actual);
}
}
}
private static void assertEqualEnvelopes(Envelope expected, Envelope actual) {
Assert.assertTrue(expected.getSequenceNumber() == actual.getSequenceNumber() &&
expected.getJobID().equals(actual.getJobID()) &&
expected.getSource().equals(actual.getSource()));
if (expected.getBuffer() == null) {
Assert.assertNull(actual.getBuffer());
}
else {
Assert.assertNotNull(actual.getBuffer());
ByteBuffer expectedByteBuffer = expected.getBuffer().getMemorySegment().wrap(0, expected.getBuffer().size());
ByteBuffer actualByteBuffer = actual.getBuffer().getMemorySegment().wrap(0, actual.getBuffer().size());
Assert.assertEquals(0, expectedByteBuffer.compareTo(actualByteBuffer));
}
if (expected.getEventsSerialized() == null) {
Assert.assertNull(actual.getEventsSerialized());
}
else {
Assert.assertNotNull(actual.getEventsSerialized());
// this is needed, because the encoding of the byte buffer
// alters the state of the buffer
expected.getEventsSerialized().clear();
List<? extends AbstractEvent> expectedEvents = expected.deserializeEvents();
List<? extends AbstractEvent> actualEvents = actual.deserializeEvents();
Assert.assertEquals(expectedEvents.size(), actualEvents.size());
for (int i = 0; i < expectedEvents.size(); i++) {
AbstractEvent expectedEvent = expectedEvents.get(i);
AbstractEvent actualEvent = actualEvents.get(i);
Assert.assertEquals(expectedEvent.getClass(), actualEvent.getClass());
Assert.assertEquals(expectedEvent, actualEvent);
}
}
}
private static ByteBuf[] randomSlices(ByteBuf buf) {
List<Integer> sliceSizes = new LinkedList<Integer>();
if (buf.readableBytes() < MIN_SLICE_SIZE) {
throw new IllegalStateException("Buffer to slice is smaller than required minimum slice size");
}
int available = buf.readableBytes() - MIN_SLICE_SIZE;
while (available > 0) {
int size = Math.min(available, Math.max(MIN_SLICE_SIZE, random.nextInt(MAX_SLICE_SIZE) + 1));
available -= size;
sliceSizes.add(size);
}
int[] slices = new int[sliceSizes.size()];
for (int i = 0; i < sliceSizes.size(); i++) {
slices[i] = sliceSizes.get(i);
}
return slice(buf, slices);
}
/**
* Returns slices with the specified sizes of the given buffer.
* <p/>
* When given n indexes, n+1 slices will be returned:
* <ul>
* <li>0 - sliceSizes[0]</li>
* <li>sliceSizes[0] - sliceSizes[1]</li>
* <li>...</li>
* <li>sliceSizes[n-1] - buf.capacity()</li>
* </ul>
*
* @return slices with the specified sizes of the given buffer
*/
private static ByteBuf[] slice(ByteBuf buf, int... sliceSizes) {
if (sliceSizes.length == 0) {
throw new IllegalStateException("Need to provide at least one slice size");
}
int numSlices = sliceSizes.length;
// transform slice sizes to buffer indexes
for (int i = 1; i < numSlices; i++) {
sliceSizes[i] += sliceSizes[i - 1];
}
for (int i = 0; i < sliceSizes.length - 1; i++) {
if (sliceSizes[i] >= sliceSizes[i + 1] || sliceSizes[i] <= 0 || sliceSizes[i] >= buf.capacity()) {
throw new IllegalStateException(
String.format("Slice size %s are off for %s", Arrays.toString(sliceSizes), buf));
}
}
ByteBuf[] slices = new ByteBuf[numSlices + 1];
// slice at slice indexes
slices[0] = buf.slice(0, sliceSizes[0]).retain();
for (int i = 1; i < numSlices; i++) {
slices[i] = buf.slice(sliceSizes[i - 1], sliceSizes[i] - sliceSizes[i - 1]).retain();
}
slices[numSlices] = buf.slice(sliceSizes[numSlices - 1], buf.capacity() - sliceSizes[numSlices - 1]).retain();
return slices;
}
// ------------------------------------------------------------------------
// mocking
// ------------------------------------------------------------------------
private static JobID anyJobId() {
return Matchers.anyObject();
}
private static ChannelID anyChannelId() {
return Matchers.anyObject();
}
// these following two Answer classes are quite ugly, but they allow to implement a randomized
// test of encoding and decoding envelopes
private static class RandomBufferRequestAnswer implements Answer<Buffer> {
private final Random random;
private boolean forced;
private RandomBufferRequestAnswer(Random random) {
this.random = random;
}
@Override
public Buffer answer(InvocationOnMock invocation) throws Throwable {
if (this.forced) {
Buffer toReturn = allocBuffer((Integer) invocation.getArguments()[0]);
this.forced = false;
return toReturn;
}
return this.random.nextBoolean() ? allocBuffer((Integer) invocation.getArguments()[0]) : null;
}
public void forceBufferAvailable() {
this.forced = true;
}
}
private static class RandomBufferAvailabilityRegistrationAnswer implements Answer<BufferAvailabilityRegistration> {
private final Random random;
private final RandomBufferRequestAnswer bufferRequestAnswer;
private boolean isRegistered = false;
private int numSkipped;
private RandomBufferAvailabilityRegistrationAnswer(Random random, RandomBufferRequestAnswer bufferRequestAnswer) {
this.random = random;
this.bufferRequestAnswer = bufferRequestAnswer;
}
@Override
public BufferAvailabilityRegistration answer(InvocationOnMock invocation) throws Throwable {
if (this.random.nextBoolean()) {
this.isRegistered = true;
return BufferAvailabilityRegistration.REGISTERED;
}
else if (this.random.nextBoolean()) {
this.bufferRequestAnswer.forceBufferAvailable();
return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_AVAILABLE;
}
else {
this.numSkipped++;
return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
}
}
public Envelope[] removeSkippedEnvelopes(Envelope[] envelopes) {
this.random.setSeed(RANDOM_SEED);
Envelope[] envelopesWithoutSkipped = new Envelope[envelopes.length - this.numSkipped];
int numEnvelopes = 0;
for (Envelope env : envelopes) {
if (env.getBuffer() != null) {
// skip envelope if returned NOT_REGISTERED_BUFFER_POOL_DESTROYED
if (!this.random.nextBoolean() && !this.random.nextBoolean() && !this.random.nextBoolean()) {
continue;
}
}
envelopesWithoutSkipped[numEnvelopes++] = env;
}
return envelopesWithoutSkipped;
}
public boolean isRegistered() {
return this.isRegistered;
}
public void unregister() {
this.isRegistered = false;
}
}
// ------------------------------------------------------------------------
public static final class TestEvent1 extends AbstractEvent {
private long id;
public TestEvent1() {
}
public TestEvent1(long id) {
this.id = id;
}
@Override
public void write(DataOutput out) throws IOException {
out.writeLong(id);
}
@Override
public void read(DataInput in) throws IOException {
id = in.readLong();
}
@Override
public boolean equals(Object obj) {
return obj.getClass() == TestEvent1.class && ((TestEvent1) obj).id == this.id;
}
@Override
public int hashCode() {
return ((int) id) ^ ((int) (id >>> 32));
}
@Override
public String toString() {
return "TestEvent1 (" + id + ")";
}
}
public static final class TestEvent2 extends AbstractEvent {
private long id;
public TestEvent2() {
}
public TestEvent2(long id) {
this.id = id;
}
@Override
public void write(DataOutput out) throws IOException {
out.writeLong(id);
}
@Override
public void read(DataInput in) throws IOException {
id = in.readLong();
}
@Override
public boolean equals(Object obj) {
return obj.getClass() == TestEvent2.class && ((TestEvent2) obj).id == this.id;
}
@Override
public int hashCode() {
return ((int) id) ^ ((int) (id >>> 32));
}
@Override
public String toString() {
return "TestEvent2 (" + id + ")";
}
}
}
/***********************************************************************************************************************
* Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network.netty;
import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.runtime.io.network.ChannelManager;
import eu.stratosphere.runtime.io.network.Envelope;
import eu.stratosphere.runtime.io.network.RemoteReceiver;
import junit.framework.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.mockito.Matchers;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import java.io.IOException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.util.Arrays;
import java.util.Collection;
import java.util.Random;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
@RunWith(Parameterized.class)
public class NettyConnectionManagerTest {
private final static long RANDOM_SEED = 520346508276087l;
private final static Random random = new Random(RANDOM_SEED);
private final static int BIND_PORT = 20000;
private final static int HIGH_WATERMARK = 32 * 1024;
private int numSubtasks;
private int numToSendPerSubtask;
private int numInThreads;
private int numOutThreads;
private int numChannels;
public NettyConnectionManagerTest(int numSubtasks, int numToSendPerSubtask, int numChannels, int numInThreads, int numOutThreads) {
this.numSubtasks = numSubtasks;
this.numToSendPerSubtask = numToSendPerSubtask;
this.numChannels = numChannels;
this.numInThreads = numInThreads;
this.numOutThreads = numOutThreads;
}
@Parameterized.Parameters
public static Collection configure() {
return Arrays.asList(
new Object[][]{
{64, 4096, 1, 1, 1},
{128, 2048, 1, 1, 1},
{256, 1024, 1, 1, 1},
{512, 512, 1, 1, 1},
{64, 4096, 4, 1, 1},
{128, 2048, 4, 1, 1},
{256, 1024, 4, 1, 1},
{512, 512, 4, 1, 1},
{64, 4096, 4, 2, 2},
{128, 2048, 4, 2, 2},
{256, 1024, 4, 2, 2},
{512, 512, 4, 2, 2}
}
);
}
@Test
public void testEnqueueRaceAndDeadlockFreeMultipleChannels() throws Exception {
final InetAddress localhost = InetAddress.getLocalHost();
final CountDownLatch latch = new CountDownLatch(this.numSubtasks);
// --------------------------------------------------------------------
// setup
// --------------------------------------------------------------------
ChannelManager channelManager = mock(ChannelManager.class);
doAnswer(new VerifyEnvelopes(latch)).when(channelManager).dispatchFromNetwork(Matchers.<Envelope>anyObject());
NettyConnectionManager connManagerToTest = new NettyConnectionManager(channelManager, localhost,
BIND_PORT, HIGH_WATERMARK, this.numInThreads, this.numOutThreads, -1, -1);
NettyConnectionManager connManagerReceiver = new NettyConnectionManager(channelManager, localhost,
BIND_PORT + 1, HIGH_WATERMARK, this.numInThreads, this.numOutThreads, -1, -1);
// --------------------------------------------------------------------
// start sender threads
// --------------------------------------------------------------------
RemoteReceiver[] receivers = new RemoteReceiver[this.numChannels];
for (int i = 0; i < this.numChannels; i++) {
receivers[i] = new RemoteReceiver(new InetSocketAddress(localhost, BIND_PORT + 1), i);
}
for (int i = 0; i < this.numSubtasks; i++) {
RemoteReceiver receiver = receivers[random.nextInt(this.numChannels)];
new Thread(new SubtaskSenderThread(connManagerToTest, receiver)).start();
}
latch.await();
connManagerToTest.shutdown();
connManagerReceiver.shutdown();
}
private class VerifyEnvelopes implements Answer {
private final ConcurrentMap<ChannelID, Integer> received = new ConcurrentHashMap<ChannelID, Integer>();
private final CountDownLatch latch;
private VerifyEnvelopes(CountDownLatch latch) {
this.latch = latch;
}
@Override
public Object answer(InvocationOnMock invocation) throws Throwable {
Envelope env = (Envelope) invocation.getArguments()[0];
ChannelID channelId = env.getSource();
int seqNum = env.getSequenceNumber();
if (seqNum == 0) {
Assert.assertNull(
String.format("Received envelope from %s before, but current seq num is 0", channelId),
this.received.putIfAbsent(channelId, seqNum));
}
else {
Assert.assertTrue(
String.format("Received seq num %d from %s, but previous was not %d", seqNum, channelId, seqNum - 1),
this.received.replace(channelId, seqNum - 1, seqNum));
}
// count down the latch if all envelopes received for this source
if (seqNum == numToSendPerSubtask - 1) {
this.latch.countDown();
}
return null;
}
}
private class SubtaskSenderThread implements Runnable {
private final NettyConnectionManager connectionManager;
private final RemoteReceiver receiver;
private final JobID jobId = new JobID();
private final ChannelID channelId = new ChannelID();
private int seqNum = 0;
private SubtaskSenderThread(NettyConnectionManager connectionManager, RemoteReceiver receiver) {
this.connectionManager = connectionManager;
this.receiver = receiver;
}
@Override
public void run() {
// enqueue envelopes with ascending seq nums
while (this.seqNum < numToSendPerSubtask) {
try {
Envelope env = new Envelope(this.seqNum++, this.jobId, this.channelId);
this.connectionManager.enqueue(env, receiver);
} catch (IOException e) {
throw new RuntimeException("Unexpected exception while enqueing envelope");
}
}
}
}
}
/***********************************************************************************************************************
* Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.runtime.io.network.netty;
import eu.stratosphere.core.memory.MemorySegment;
import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.runtime.io.network.Envelope;
import io.netty.buffer.ByteBuf;
import io.netty.channel.embedded.EmbeddedChannel;
import junit.framework.Assert;
import org.junit.Test;
import java.nio.ByteBuffer;
import java.util.Random;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
public class OutboundEnvelopeEncoderTest {
private final long RANDOM_SEED = 520346508276087l;
private final Random random = new Random(RANDOM_SEED);
private static final int NUM_RANDOM_ENVELOPES = 512;
private static final int MAX_EVENTS_SIZE = 1024;
private static final int MAX_BUFFER_SIZE = 32768;
@Test
public void testEncodedSizeAndBufferRecycling() {
final ByteBuffer events = ByteBuffer.allocate(MAX_EVENTS_SIZE);
final MemorySegment segment = new MemorySegment(new byte[MAX_BUFFER_SIZE]);
final Buffer buffer = mock(Buffer.class);
when(buffer.getMemorySegment()).thenReturn(segment);
final EmbeddedChannel channel = new EmbeddedChannel(new OutboundEnvelopeEncoder());
int numBuffers = 0;
for (int i = 0; i < NUM_RANDOM_ENVELOPES; i++) {
Envelope env = new Envelope(i, new JobID(), new ChannelID());
int expectedEncodedMsgSize = OutboundEnvelopeEncoder.HEADER_SIZE;
if (random.nextBoolean()) {
int eventsSize = random.nextInt(MAX_EVENTS_SIZE + 1);
expectedEncodedMsgSize += eventsSize;
events.clear();
events.limit(eventsSize);
env.setEventsSerialized(events);
}
if (random.nextBoolean()) {
numBuffers++;
int bufferSize = random.nextInt(MAX_BUFFER_SIZE + 1);
when(buffer.size()).thenReturn(bufferSize);
env.setBuffer(buffer);
expectedEncodedMsgSize += bufferSize;
}
Assert.assertTrue(channel.writeOutbound(env));
// --------------------------------------------------------------------
// verify encoded ByteBuf size
// --------------------------------------------------------------------
ByteBuf encodedMsg = (ByteBuf) channel.readOutbound();
Assert.assertEquals(expectedEncodedMsgSize, encodedMsg.readableBytes());
encodedMsg.release();
}
// --------------------------------------------------------------------
// verify buffers are recycled
// --------------------------------------------------------------------
verify(buffer, times(numBuffers)).recycleBuffer();
}
}
...@@ -14,6 +14,7 @@ ...@@ -14,6 +14,7 @@
**********************************************************************************************************************/ **********************************************************************************************************************/
package eu.stratosphere.test.broadcastvars; package eu.stratosphere.test.broadcastvars;
import eu.stratosphere.nephele.jobgraph.DistributionPattern;
import org.apache.log4j.Level; import org.apache.log4j.Level;
import eu.stratosphere.api.common.operators.util.UserCodeObjectWrapper; import eu.stratosphere.api.common.operators.util.UserCodeObjectWrapper;
...@@ -22,8 +23,6 @@ import eu.stratosphere.api.common.typeutils.TypeSerializerFactory; ...@@ -22,8 +23,6 @@ import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
import eu.stratosphere.api.java.record.io.CsvInputFormat; import eu.stratosphere.api.java.record.io.CsvInputFormat;
import eu.stratosphere.configuration.Configuration; import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.core.fs.Path; import eu.stratosphere.core.fs.Path;
import eu.stratosphere.nephele.io.DistributionPattern;
import eu.stratosphere.nephele.io.channels.ChannelType;
import eu.stratosphere.nephele.jobgraph.JobGraph; import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException; import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
import eu.stratosphere.nephele.jobgraph.JobInputVertex; import eu.stratosphere.nephele.jobgraph.JobInputVertex;
......
...@@ -23,7 +23,6 @@ import eu.stratosphere.client.LocalExecutor; ...@@ -23,7 +23,6 @@ import eu.stratosphere.client.LocalExecutor;
import eu.stratosphere.test.recordJobs.wordcount.WordCount; import eu.stratosphere.test.recordJobs.wordcount.WordCount;
import eu.stratosphere.test.testdata.WordCountData; import eu.stratosphere.test.testdata.WordCountData;
public class LocalExecutorITCase { public class LocalExecutorITCase {
@Test @Test
......
...@@ -28,7 +28,6 @@ public class WordCountITCase extends JavaProgramTestBase { ...@@ -28,7 +28,6 @@ public class WordCountITCase extends JavaProgramTestBase {
setNumTaskManager(2); setNumTaskManager(2);
} }
@Override @Override
protected void preSubmit() throws Exception { protected void preSubmit() throws Exception {
textPath = createTempFile("text.txt", WordCountData.TEXT); textPath = createTempFile("text.txt", WordCountData.TEXT);
......
...@@ -16,6 +16,8 @@ package eu.stratosphere.test.iterative.nephele; ...@@ -16,6 +16,8 @@ package eu.stratosphere.test.iterative.nephele;
import java.io.BufferedReader; import java.io.BufferedReader;
import java.util.Collection; import java.util.Collection;
import eu.stratosphere.nephele.jobgraph.DistributionPattern;
import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.test.util.RecordAPITestBase; import eu.stratosphere.test.util.RecordAPITestBase;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.junit.runners.Parameterized; import org.junit.runners.Parameterized;
...@@ -31,8 +33,6 @@ import eu.stratosphere.api.java.record.io.CsvInputFormat; ...@@ -31,8 +33,6 @@ import eu.stratosphere.api.java.record.io.CsvInputFormat;
import eu.stratosphere.api.java.record.io.CsvOutputFormat; import eu.stratosphere.api.java.record.io.CsvOutputFormat;
import eu.stratosphere.api.java.record.io.FileOutputFormat; import eu.stratosphere.api.java.record.io.FileOutputFormat;
import eu.stratosphere.configuration.Configuration; import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.nephele.io.DistributionPattern;
import eu.stratosphere.nephele.io.channels.ChannelType;
import eu.stratosphere.nephele.jobgraph.JobGraph; import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException; import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
import eu.stratosphere.nephele.jobgraph.JobInputVertex; import eu.stratosphere.nephele.jobgraph.JobInputVertex;
...@@ -425,11 +425,11 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase { ...@@ -425,11 +425,11 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
JobGraphUtils.connect(head, intermediate, ChannelType.NETWORK, DistributionPattern.BIPARTITE); JobGraphUtils.connect(head, intermediate, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, numSubTasks); intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, numSubTasks);
JobGraphUtils.connect(intermediate, tail, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(intermediate, tail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1); tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(tail, fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(tail, fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE); JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
...@@ -567,16 +567,16 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase { ...@@ -567,16 +567,16 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
JobGraphUtils.connect(intermediate, ssJoinIntermediate, ChannelType.NETWORK, DistributionPattern.POINTWISE); JobGraphUtils.connect(intermediate, ssJoinIntermediate, ChannelType.NETWORK, DistributionPattern.POINTWISE);
ssJoinIntermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1); ssJoinIntermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
JobGraphUtils.connect(ssJoinIntermediate, ssTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(ssJoinIntermediate, ssTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
ssTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1); ssTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
JobGraphUtils.connect(ssJoinIntermediate, wsTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(ssJoinIntermediate, wsTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
wsTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1); wsTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(ssTail, ssFakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(ssTail, ssFakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(wsTail, wsFakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(wsTail, wsFakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE); JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
...@@ -695,12 +695,12 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase { ...@@ -695,12 +695,12 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
DistributionPattern.POINTWISE); DistributionPattern.POINTWISE);
wsUpdateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1); wsUpdateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
JobGraphUtils.connect(wsUpdateIntermediate, ssTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(wsUpdateIntermediate, ssTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
ssTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1); ssTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(ssTail, fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(ssTail, fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE); JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
...@@ -815,12 +815,12 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase { ...@@ -815,12 +815,12 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
JobGraphUtils.connect(intermediate, ssJoinIntermediate, ChannelType.NETWORK, DistributionPattern.POINTWISE); JobGraphUtils.connect(intermediate, ssJoinIntermediate, ChannelType.NETWORK, DistributionPattern.POINTWISE);
ssJoinIntermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1); ssJoinIntermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
JobGraphUtils.connect(ssJoinIntermediate, wsTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(ssJoinIntermediate, wsTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
wsTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1); wsTailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(wsTail, fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(wsTail, fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE); JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
......
...@@ -15,6 +15,8 @@ package eu.stratosphere.test.iterative.nephele; ...@@ -15,6 +15,8 @@ package eu.stratosphere.test.iterative.nephele;
import java.util.Collection; import java.util.Collection;
import java.util.Iterator; import java.util.Iterator;
import eu.stratosphere.nephele.jobgraph.DistributionPattern;
import eu.stratosphere.runtime.io.channels.ChannelType;
import org.junit.runner.RunWith; import org.junit.runner.RunWith;
import org.junit.runners.Parameterized; import org.junit.runners.Parameterized;
...@@ -25,8 +27,6 @@ import eu.stratosphere.api.java.record.functions.MapFunction; ...@@ -25,8 +27,6 @@ import eu.stratosphere.api.java.record.functions.MapFunction;
import eu.stratosphere.api.java.record.functions.ReduceFunction; import eu.stratosphere.api.java.record.functions.ReduceFunction;
import eu.stratosphere.api.java.record.io.FileOutputFormat; import eu.stratosphere.api.java.record.io.FileOutputFormat;
import eu.stratosphere.configuration.Configuration; import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.nephele.io.DistributionPattern;
import eu.stratosphere.nephele.io.channels.ChannelType;
import eu.stratosphere.nephele.jobgraph.JobGraph; import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException; import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
import eu.stratosphere.nephele.jobgraph.JobInputVertex; import eu.stratosphere.nephele.jobgraph.JobInputVertex;
...@@ -232,16 +232,16 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase { ...@@ -232,16 +232,16 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
// -------------------------------------------------------------------------------------------------------------- // --------------------------------------------------------------------------------------------------------------
// 2. EDGES // 2. EDGES
// -------------------------------------------------------------------------------------------------------------- // --------------------------------------------------------------------------------------------------------------
JobGraphUtils.connect(input, head, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(input, head, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(head, tail, ChannelType.INMEMORY, DistributionPattern.BIPARTITE); JobGraphUtils.connect(head, tail, ChannelType.IN_MEMORY, DistributionPattern.BIPARTITE);
tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, numSubTasks); tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, numSubTasks);
JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE); JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
JobGraphUtils.connect(tail, fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE); JobGraphUtils.connect(tail, fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
// -------------------------------------------------------------------------------------------------------------- // --------------------------------------------------------------------------------------------------------------
// 3. INSTANCE SHARING // 3. INSTANCE SHARING
......
...@@ -22,8 +22,6 @@ import eu.stratosphere.api.java.record.io.CsvOutputFormat; ...@@ -22,8 +22,6 @@ import eu.stratosphere.api.java.record.io.CsvOutputFormat;
import eu.stratosphere.api.java.record.io.TextInputFormat; import eu.stratosphere.api.java.record.io.TextInputFormat;
import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.MapOperator;
import eu.stratosphere.api.java.record.operators.ReduceOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator;
import eu.stratosphere.nephele.io.MutableUnionRecordReader;
import eu.stratosphere.nephele.io.UnionRecordReader;
import eu.stratosphere.test.recordJobs.wordcount.WordCount.CountWords; import eu.stratosphere.test.recordJobs.wordcount.WordCount.CountWords;
import eu.stratosphere.test.recordJobs.wordcount.WordCount.TokenizeLine; import eu.stratosphere.test.recordJobs.wordcount.WordCount.TokenizeLine;
import eu.stratosphere.test.testdata.WordCountData; import eu.stratosphere.test.testdata.WordCountData;
...@@ -39,8 +37,6 @@ import eu.stratosphere.types.StringValue; ...@@ -39,8 +37,6 @@ import eu.stratosphere.types.StringValue;
* *
* @see {@link https://github.com/stratosphere/stratosphere/issues/192} * @see {@link https://github.com/stratosphere/stratosphere/issues/192}
* @see {@link https://github.com/stratosphere/stratosphere/issues/124} * @see {@link https://github.com/stratosphere/stratosphere/issues/124}
* @see {@link UnionRecordReader}
* @see {@link MutableUnionRecordReader}
*/ */
public class WordCountUnionReduceITCase extends RecordAPITestBase { public class WordCountUnionReduceITCase extends RecordAPITestBase {
......
/***********************************************************************************************************************
* Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.test.runtime;
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.core.io.IOReadableWritable;
import eu.stratosphere.nephele.jobgraph.DistributionPattern;
import eu.stratosphere.nephele.jobgraph.JobGenericInputVertex;
import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
import eu.stratosphere.nephele.jobgraph.JobInputVertex;
import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
import eu.stratosphere.nephele.template.AbstractGenericInputTask;
import eu.stratosphere.nephele.template.AbstractOutputTask;
import eu.stratosphere.nephele.template.AbstractTask;
import eu.stratosphere.runtime.io.api.RecordReader;
import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.test.util.RecordAPITestBase;
import eu.stratosphere.util.LogUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.junit.After;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
@RunWith(Parameterized.class)
public class NetworkStackNepheleITCase extends RecordAPITestBase {
private static final Log LOG = LogFactory.getLog(NetworkStackNepheleITCase.class);
private static final String DATA_VOLUME_GB_CONFIG_KEY = "data.volume.gb";
private static final String USE_FORWARDER_CONFIG_KEY = "use.forwarder";
private static final String NUM_SUBTASKS_CONFIG_KEY = "num.subtasks";
private static final String NUM_SUBTASKS_PER_INSTANCE_CONFIG_KEY = "num.subtasks.instance";
private static final String IS_SLOW_SENDER_CONFIG_KEY = "is.slow.sender";
private static final String IS_SLOW_RECEIVER_CONFIG_KEY = "is.slow.receiver";
private static final int IS_SLOW_SLEEP_MS = 10;
private static final int IS_SLOW_EVERY_NUM_RECORDS = (2 * 32 * 1024) / SpeedTestRecord.RECORD_SIZE;
// ------------------------------------------------------------------------
public NetworkStackNepheleITCase(Configuration config) {
super(config);
setNumTaskManager(2);
LogUtils.initializeDefaultConsoleLogger();
}
@Parameters
public static Collection<Object[]> getConfigurations() {
Object[][] configParams = new Object[][]{
new Object[]{1, false, false, false, 4, 2},
new Object[]{1, true, false, false, 4, 2},
new Object[]{1, true, true, false, 4, 2},
new Object[]{1, true, false, true, 4, 2},
new Object[]{2, true, false, false, 4, 2},
new Object[]{4, true, false, false, 4, 2},
new Object[]{4, true, false, false, 8, 4},
new Object[]{4, true, false, false, 16, 8},
};
List<Configuration> configs = new ArrayList<Configuration>(configParams.length);
for (Object[] p : configParams) {
Configuration config = new Configuration();
config.setInteger(DATA_VOLUME_GB_CONFIG_KEY, (Integer) p[0]);
config.setBoolean(USE_FORWARDER_CONFIG_KEY, (Boolean) p[1]);
config.setBoolean(IS_SLOW_SENDER_CONFIG_KEY, (Boolean) p[2]);
config.setBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, (Boolean) p[3]);
config.setInteger(NUM_SUBTASKS_CONFIG_KEY, (Integer) p[4]);
config.setInteger(NUM_SUBTASKS_PER_INSTANCE_CONFIG_KEY, (Integer) p[5]);
configs.add(config);
}
return toParameterList(configs);
}
// ------------------------------------------------------------------------
@Override
protected JobGraph getJobGraph() throws Exception {
int dataVolumeGb = this.config.getInteger(DATA_VOLUME_GB_CONFIG_KEY, 1);
boolean useForwarder = this.config.getBoolean(USE_FORWARDER_CONFIG_KEY, true);
boolean isSlowSender = this.config.getBoolean(IS_SLOW_SENDER_CONFIG_KEY, false);
boolean isSlowReceiver = this.config.getBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, false);
int numSubtasks = this.config.getInteger(NUM_SUBTASKS_CONFIG_KEY, 1);
int numSubtasksPerInstance = this.config.getInteger(NUM_SUBTASKS_PER_INSTANCE_CONFIG_KEY, 1);
return createJobGraph(dataVolumeGb, useForwarder, isSlowSender, isSlowReceiver, numSubtasks, numSubtasksPerInstance);
}
@After
public void calculateThroughput() {
if (getJobExecutionResult() != null) {
int dataVolumeGb = this.config.getInteger(DATA_VOLUME_GB_CONFIG_KEY, 1);
double dataVolumeMbit = dataVolumeGb * 8192.0;
double runtimeSecs = getJobExecutionResult().getNetRuntime() / 1000.0;
int mbitPerSecond = (int) Math.round(dataVolumeMbit / runtimeSecs);
LOG.info(String.format("Test finished with throughput of %d MBit/s (" +
"runtime [secs]: %.2f, data volume [mbits]: %.2f)", mbitPerSecond, runtimeSecs, dataVolumeMbit));
}
}
private JobGraph createJobGraph(int dataVolumeGb, boolean useForwarder, boolean isSlowSender, boolean isSlowReceiver,
int numSubtasks, int numSubtasksPerInstance) throws JobGraphDefinitionException {
JobGraph jobGraph = new JobGraph("Speed Test");
JobInputVertex producer = new JobGenericInputVertex("Speed Test Producer", jobGraph);
producer.setInputClass(SpeedTestProducer.class);
producer.setNumberOfSubtasks(numSubtasks);
producer.setNumberOfSubtasksPerInstance(numSubtasksPerInstance);
producer.getConfiguration().setInteger(DATA_VOLUME_GB_CONFIG_KEY, dataVolumeGb);
producer.getConfiguration().setBoolean(IS_SLOW_SENDER_CONFIG_KEY, isSlowSender);
JobTaskVertex forwarder = null;
if (useForwarder) {
forwarder = new JobTaskVertex("Speed Test Forwarder", jobGraph);
forwarder.setTaskClass(SpeedTestForwarder.class);
forwarder.setNumberOfSubtasks(numSubtasks);
forwarder.setNumberOfSubtasksPerInstance(numSubtasksPerInstance);
}
JobOutputVertex consumer = new JobOutputVertex("Speed Test Consumer", jobGraph);
consumer.setOutputClass(SpeedTestConsumer.class);
consumer.setNumberOfSubtasks(numSubtasks);
consumer.setNumberOfSubtasksPerInstance(numSubtasksPerInstance);
consumer.getConfiguration().setBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, isSlowReceiver);
if (useForwarder) {
producer.connectTo(forwarder, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
forwarder.connectTo(consumer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
producer.setVertexToShareInstancesWith(forwarder);
forwarder.setVertexToShareInstancesWith(consumer);
}
else {
producer.connectTo(consumer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
producer.setVertexToShareInstancesWith(consumer);
}
return jobGraph;
}
// ------------------------------------------------------------------------
public static class SpeedTestProducer extends AbstractGenericInputTask {
private RecordWriter<SpeedTestRecord> writer;
@Override
public void registerInputOutput() {
this.writer = new RecordWriter<SpeedTestRecord>(this);
}
@Override
public void invoke() throws Exception {
this.writer.initializeSerializers();
// Determine the amount of data to send per subtask
int dataVolumeGb = getTaskConfiguration().getInteger(NetworkStackNepheleITCase.DATA_VOLUME_GB_CONFIG_KEY, 1);
long dataMbPerSubtask = (dataVolumeGb * 1024) / getCurrentNumberOfSubtasks();
long numRecordsToEmit = (dataMbPerSubtask * 1024 * 1024) / SpeedTestRecord.RECORD_SIZE;
LOG.info(String.format("%d/%d: Producing %d records (each record: %d bytes, total: %.2f GB)",
getIndexInSubtaskGroup() + 1, getCurrentNumberOfSubtasks(), numRecordsToEmit,
SpeedTestRecord.RECORD_SIZE, dataMbPerSubtask/1024.0));
boolean isSlow = getTaskConfiguration().getBoolean(IS_SLOW_SENDER_CONFIG_KEY, false);
int numRecords = 0;
SpeedTestRecord record = new SpeedTestRecord();
for (long i = 0; i < numRecordsToEmit; i++) {
if (isSlow && (numRecords++ % IS_SLOW_EVERY_NUM_RECORDS) == 0) {
Thread.sleep(IS_SLOW_SLEEP_MS);
}
this.writer.emit(record);
}
this.writer.flush();
}
}
public static class SpeedTestForwarder extends AbstractTask {
private RecordReader<SpeedTestRecord> reader;
private RecordWriter<SpeedTestRecord> writer;
@Override
public void registerInputOutput() {
this.reader = new RecordReader<SpeedTestRecord>(this, SpeedTestRecord.class);
this.writer = new RecordWriter<SpeedTestRecord>(this);
}
@Override
public void invoke() throws Exception {
this.writer.initializeSerializers();
SpeedTestRecord record;
while ((record = this.reader.next()) != null) {
this.writer.emit(record);
}
this.writer.flush();
}
}
public static class SpeedTestConsumer extends AbstractOutputTask {
private RecordReader<SpeedTestRecord> reader;
@Override
public void registerInputOutput() {
this.reader = new RecordReader<SpeedTestRecord>(this, SpeedTestRecord.class);
}
@Override
public void invoke() throws Exception {
boolean isSlow = getTaskConfiguration().getBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, false);
int numRecords = 0;
while (this.reader.next() != null) {
if (isSlow && (numRecords++ % IS_SLOW_EVERY_NUM_RECORDS) == 0) {
Thread.sleep(IS_SLOW_SLEEP_MS);
}
}
}
}
public static class SpeedTestRecord implements IOReadableWritable {
private static final int RECORD_SIZE = 128;
private final byte[] buf = new byte[RECORD_SIZE];
public SpeedTestRecord() {
for (int i = 0; i < RECORD_SIZE; ++i) {
this.buf[i] = (byte) (i % 128);
}
}
@Override
public void write(DataOutput out) throws IOException {
out.write(this.buf);
}
@Override
public void read(DataInput in) throws IOException {
in.readFully(this.buf);
}
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册