You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by rm...@apache.org on 2014/06/09 20:30:36 UTC
[01/30] git commit: Fixes bugs where the TypeExtractor throws an NPE
instead of the operators
Repository: incubator-flink
Updated Branches:
refs/heads/master b746f452e -> 24d00598a
Fixes bugs where the TypeExtractor throws an NPE instead of the operators
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/2b0baea9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/2b0baea9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/2b0baea9
Branch: refs/heads/master
Commit: 2b0baea9b8a6dd99052c2dfa98cae719a39d6bbc
Parents: b746f45
Author: twalthr <in...@twalthr.com>
Authored: Mon Jun 2 14:39:36 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Fri Jun 6 15:53:24 2014 +0200
----------------------------------------------------------------------
.../main/java/eu/stratosphere/api/java/DataSet.java | 15 +++++++++++++++
.../api/java/operators/CoGroupOperator.java | 3 +++
.../api/java/operators/CrossOperator.java | 3 +++
.../api/java/operators/FilterOperator.java | 4 ----
.../api/java/operators/FlatMapOperator.java | 4 ----
.../api/java/operators/JoinOperator.java | 3 +++
.../stratosphere/api/java/operators/MapOperator.java | 4 ----
.../api/java/operators/ReduceGroupOperator.java | 8 --------
.../api/java/operators/ReduceOperator.java | 8 --------
.../api/java/operators/SortedGrouping.java | 3 +++
.../api/java/operators/UnsortedGrouping.java | 6 ++++++
11 files changed, 33 insertions(+), 28 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java
index 66f8eeb..758cbf2 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/DataSet.java
@@ -130,6 +130,9 @@ public abstract class DataSet<T> {
* @see DataSet
*/
public <R> MapOperator<T, R> map(MapFunction<T, R> mapper) {
+ if (mapper == null) {
+ throw new NullPointerException("Map function must not be null.");
+ }
return new MapOperator<T, R>(this, mapper);
}
@@ -146,6 +149,9 @@ public abstract class DataSet<T> {
* @see DataSet
*/
public <R> FlatMapOperator<T, R> flatMap(FlatMapFunction<T, R> flatMapper) {
+ if (flatMapper == null) {
+ throw new NullPointerException("FlatMap function must not be null.");
+ }
return new FlatMapOperator<T, R>(this, flatMapper);
}
@@ -163,6 +169,9 @@ public abstract class DataSet<T> {
* @see DataSet
*/
public FilterOperator<T> filter(FilterFunction<T> filter) {
+ if (filter == null) {
+ throw new NullPointerException("Filter function must not be null.");
+ }
return new FilterOperator<T>(this, filter);
}
@@ -229,6 +238,9 @@ public abstract class DataSet<T> {
* @see DataSet
*/
public ReduceOperator<T> reduce(ReduceFunction<T> reducer) {
+ if (reducer == null) {
+ throw new NullPointerException("Reduce function must not be null.");
+ }
return new ReduceOperator<T>(this, reducer);
}
@@ -246,6 +258,9 @@ public abstract class DataSet<T> {
* @see DataSet
*/
public <R> ReduceGroupOperator<T, R> reduceGroup(GroupReduceFunction<T, R> reducer) {
+ if (reducer == null) {
+ throw new NullPointerException("GroupReduce function must not be null.");
+ }
return new ReduceGroupOperator<T, R>(this, reducer);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java
index 67b9e5c..ca4b1db 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CoGroupOperator.java
@@ -439,6 +439,9 @@ public class CoGroupOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OU
* @see DataSet
*/
public <R> CoGroupOperator<I1, I2, R> with(CoGroupFunction<I1, I2, R> function) {
+ if (function == null) {
+ throw new NullPointerException("CoGroup function must not be null.");
+ }
TypeInformation<R> returnType = TypeExtractor.getCoGroupReturnTypes(function, input1.getType(), input2.getType());
return new CoGroupOperator<I1, I2, R>(input1, input2, keys1, keys2, function, returnType);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java
index 48da306..3566224 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/CrossOperator.java
@@ -113,6 +113,9 @@ public class CrossOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1, I2, OUT,
* @see DataSet
*/
public <R> CrossOperator<I1, I2, R> with(CrossFunction<I1, I2, R> function) {
+ if (function == null) {
+ throw new NullPointerException("Cross function must not be null.");
+ }
TypeInformation<R> returnType = TypeExtractor.getCrossReturnTypes(function, input1.getType(), input2.getType());
return new CrossOperator<I1, I2, R>(input1, input2, function, returnType);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FilterOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FilterOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FilterOperator.java
index d9db757..adbe77f 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FilterOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FilterOperator.java
@@ -34,10 +34,6 @@ public class FilterOperator<T> extends SingleInputUdfOperator<T, T, FilterOperat
public FilterOperator(DataSet<T> input, FilterFunction<T> function) {
super(input, input.getType());
- if (function == null) {
- throw new NullPointerException("Filter function must not be null.");
- }
-
this.function = function;
extractSemanticAnnotationsFromUdf(function.getClass());
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java
index 27025c7..32f2343 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/FlatMapOperator.java
@@ -37,10 +37,6 @@ public class FlatMapOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT, Fl
public FlatMapOperator(DataSet<IN> input, FlatMapFunction<IN, OUT> function) {
super(input, TypeExtractor.getFlatMapReturnTypes(function, input.getType()));
- if (function == null) {
- throw new NullPointerException("FlatMap function must not be null.");
- }
-
this.function = function;
extractSemanticAnnotationsFromUdf(function.getClass());
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java
index 578d68c..992cc0a 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/JoinOperator.java
@@ -421,6 +421,9 @@ public abstract class JoinOperator<I1, I2, OUT> extends TwoInputUdfOperator<I1,
* @see DataSet
*/
public <R> EquiJoin<I1, I2, R> with(JoinFunction<I1, I2, R> function) {
+ if (function == null) {
+ throw new NullPointerException("Join function must not be null.");
+ }
TypeInformation<R> returnType = TypeExtractor.getJoinReturnTypes(function, getInput1Type(), getInput2Type());
return new EquiJoin<I1, I2, R>(getInput1(), getInput2(), getKeys1(), getKeys2(), function, returnType, getJoinHint());
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java
index e2c36bf..00bbb27 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/MapOperator.java
@@ -39,10 +39,6 @@ public class MapOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT, MapOpe
public MapOperator(DataSet<IN> input, MapFunction<IN, OUT> function) {
super(input, TypeExtractor.getMapReturnTypes(function, input.getType()));
- if (function == null) {
- throw new NullPointerException("Map function must not be null.");
- }
-
this.function = function;
extractSemanticAnnotationsFromUdf(function.getClass());
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java
index 9029b09..e001d91 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceGroupOperator.java
@@ -58,10 +58,6 @@ public class ReduceGroupOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT
public ReduceGroupOperator(DataSet<IN> input, GroupReduceFunction<IN, OUT> function) {
super(input, TypeExtractor.getGroupReduceReturnTypes(function, input.getType()));
- if (function == null) {
- throw new NullPointerException("GroupReduce function must not be null.");
- }
-
this.function = function;
this.grouper = null;
checkCombinability();
@@ -76,10 +72,6 @@ public class ReduceGroupOperator<IN, OUT> extends SingleInputUdfOperator<IN, OUT
public ReduceGroupOperator(Grouping<IN> input, GroupReduceFunction<IN, OUT> function) {
super(input != null ? input.getDataSet() : null, TypeExtractor.getGroupReduceReturnTypes(function, input.getDataSet().getType()));
- if (function == null) {
- throw new NullPointerException("GroupReduce function must not be null.");
- }
-
this.function = function;
this.grouper = input;
checkCombinability();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java
index bdf9436..6056e8b 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/ReduceOperator.java
@@ -53,10 +53,6 @@ public class ReduceOperator<IN> extends SingleInputUdfOperator<IN, IN, ReduceOpe
public ReduceOperator(DataSet<IN> input, ReduceFunction<IN> function) {
super(input, input.getType());
- if (function == null) {
- throw new NullPointerException("Reduce function must not be null.");
- }
-
this.function = function;
this.grouper = null;
@@ -67,10 +63,6 @@ public class ReduceOperator<IN> extends SingleInputUdfOperator<IN, IN, ReduceOpe
public ReduceOperator(Grouping<IN> input, ReduceFunction<IN> function) {
super(input.getDataSet(), input.getDataSet().getType());
- if (function == null) {
- throw new NullPointerException("Reduce function must not be null.");
- }
-
this.function = function;
this.grouper = input;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java
index 3edb05c..dc26a2b 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/SortedGrouping.java
@@ -72,6 +72,9 @@ public class SortedGrouping<T> extends Grouping<T> {
* @see DataSet
*/
public <R> ReduceGroupOperator<T, R> reduceGroup(GroupReduceFunction<T, R> reducer) {
+ if (reducer == null) {
+ throw new NullPointerException("GroupReduce function must not be null.");
+ }
return new ReduceGroupOperator<T, R>(this, reducer);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2b0baea9/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/UnsortedGrouping.java
----------------------------------------------------------------------
diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/UnsortedGrouping.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/UnsortedGrouping.java
index 711bc24..95e40bc 100644
--- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/UnsortedGrouping.java
+++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/operators/UnsortedGrouping.java
@@ -64,6 +64,9 @@ public class UnsortedGrouping<T> extends Grouping<T> {
* @see DataSet
*/
public ReduceOperator<T> reduce(ReduceFunction<T> reducer) {
+ if (reducer == null) {
+ throw new NullPointerException("Reduce function must not be null.");
+ }
return new ReduceOperator<T>(this, reducer);
}
@@ -81,6 +84,9 @@ public class UnsortedGrouping<T> extends Grouping<T> {
* @see DataSet
*/
public <R> ReduceGroupOperator<T, R> reduceGroup(GroupReduceFunction<T, R> reducer) {
+ if (reducer == null) {
+ throw new NullPointerException("GroupReduce function must not be null.");
+ }
return new ReduceGroupOperator<T, R>(this, reducer);
}
[10/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
index 965a5aa..06c857e 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/JobManagerITCase.java
@@ -13,44 +13,46 @@
package eu.stratosphere.nephele.jobmanager;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
import eu.stratosphere.configuration.ConfigConstants;
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.configuration.GlobalConfiguration;
import eu.stratosphere.core.fs.Path;
import eu.stratosphere.nephele.client.JobClient;
import eu.stratosphere.nephele.client.JobExecutionException;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.library.FileLineReader;
-import eu.stratosphere.nephele.io.library.FileLineWriter;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.nephele.jobgraph.JobFileInputVertex;
import eu.stratosphere.nephele.jobgraph.JobFileOutputVertex;
import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
import eu.stratosphere.nephele.jobmanager.JobManager.ExecutionMode;
+import eu.stratosphere.nephele.taskmanager.Task;
import eu.stratosphere.nephele.taskmanager.TaskManager;
-import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask;
+import eu.stratosphere.nephele.util.FileLineReader;
+import eu.stratosphere.nephele.util.FileLineWriter;
import eu.stratosphere.nephele.util.JarFileCreator;
import eu.stratosphere.nephele.util.ServerTestUtils;
import eu.stratosphere.util.LogUtils;
+import eu.stratosphere.util.StringUtils;
+import org.apache.log4j.ConsoleAppender;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.apache.log4j.PatternLayout;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
/**
* This test is intended to cover the basic functionality of the {@link JobManager}.
@@ -185,8 +187,8 @@ public class JobManagerITCase {
// connect vertices
try {
i1.connectTo(t1, ChannelType.NETWORK);
- t1.connectTo(t2, ChannelType.INMEMORY);
- t2.connectTo(o1, ChannelType.INMEMORY);
+ t1.connectTo(t2, ChannelType.IN_MEMORY);
+ t2.connectTo(o1, ChannelType.IN_MEMORY);
} catch (JobGraphDefinitionException e) {
e.printStackTrace();
}
@@ -286,8 +288,8 @@ public class JobManagerITCase {
o1.setVertexToShareInstancesWith(i1);
// connect vertices
- i1.connectTo(t1, ChannelType.INMEMORY);
- t1.connectTo(o1, ChannelType.INMEMORY);
+ i1.connectTo(t1, ChannelType.IN_MEMORY);
+ t1.connectTo(o1, ChannelType.IN_MEMORY);
// add jar
jg.addJar(new Path(new File(ServerTestUtils.getTempDir() + File.separator + exceptionClassName + ".jar")
@@ -297,7 +299,7 @@ public class JobManagerITCase {
jobClient = new JobClient(jg, configuration);
// deactivate logging of expected test exceptions
- Logger rtLogger = Logger.getLogger(RuntimeTask.class);
+ Logger rtLogger = Logger.getLogger(Task.class);
Level rtLevel = rtLogger.getEffectiveLevel();
rtLogger.setLevel(Level.OFF);
@@ -382,8 +384,8 @@ public class JobManagerITCase {
o1.setVertexToShareInstancesWith(i1);
// connect vertices
- i1.connectTo(t1, ChannelType.INMEMORY);
- t1.connectTo(o1, ChannelType.INMEMORY);
+ i1.connectTo(t1, ChannelType.IN_MEMORY);
+ t1.connectTo(o1, ChannelType.IN_MEMORY);
// add jar
jg.addJar(new Path(new File(ServerTestUtils.getTempDir() + File.separator + runtimeExceptionClassName
@@ -492,8 +494,8 @@ public class JobManagerITCase {
// connect vertices
try {
i1.connectTo(t1, ChannelType.NETWORK);
- t1.connectTo(t2, ChannelType.INMEMORY);
- t2.connectTo(o1, ChannelType.INMEMORY);
+ t1.connectTo(t2, ChannelType.IN_MEMORY);
+ t2.connectTo(o1, ChannelType.IN_MEMORY);
} catch (JobGraphDefinitionException e) {
e.printStackTrace();
}
@@ -583,9 +585,9 @@ public class JobManagerITCase {
o1.setVertexToShareInstancesWith(i1);
// connect vertices
- i1.connectTo(t1, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ i1.connectTo(t1, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
i1.connectTo(t1, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
- t1.connectTo(o1, ChannelType.INMEMORY);
+ t1.connectTo(o1, ChannelType.IN_MEMORY);
// add jar
jg.addJar(new Path(jarFile.toURI()));
@@ -657,7 +659,7 @@ public class JobManagerITCase {
o1.setVertexToShareInstancesWith(i1);
// connect vertices
- i1.connectTo(o1, ChannelType.INMEMORY);
+ i1.connectTo(o1, ChannelType.IN_MEMORY);
// add jar
jg.addJar(new Path(jarFile.toURI()));
@@ -751,9 +753,9 @@ public class JobManagerITCase {
u1.setVertexToShareInstancesWith(o1);
// connect vertices
- i1.connectTo(u1, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
- i2.connectTo(u1, ChannelType.INMEMORY);
- u1.connectTo(o1, ChannelType.INMEMORY);
+ i1.connectTo(u1, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
+ i2.connectTo(u1, ChannelType.IN_MEMORY);
+ u1.connectTo(o1, ChannelType.IN_MEMORY);
// add jar
jg.addJar(new Path(jarFile.toURI()));
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/UnionTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/UnionTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/UnionTask.java
index 3b95133..124a24d 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/UnionTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/UnionTask.java
@@ -14,9 +14,9 @@
package eu.stratosphere.nephele.jobmanager;
import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.MutableRecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.io.UnionRecordReader;
+import eu.stratosphere.runtime.io.api.MutableRecordReader;
+import eu.stratosphere.runtime.io.api.RecordWriter;
+import eu.stratosphere.runtime.io.api.UnionRecordReader;
import eu.stratosphere.nephele.template.AbstractTask;
/**
@@ -41,13 +41,17 @@ public class UnionTask extends AbstractTask {
recordReaders[1] = new MutableRecordReader<StringRecord>(this);
this.unionReader = new UnionRecordReader<StringRecord>(recordReaders, StringRecord.class);
- this.writer = new RecordWriter<StringRecord>(this, StringRecord.class);
+ this.writer = new RecordWriter<StringRecord>(this);
}
@Override
public void invoke() throws Exception {
+ this.writer.initializeSerializers();
+
while (this.unionReader.hasNext()) {
this.writer.emit(this.unionReader.next());
}
+
+ this.writer.flush();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueSchedulerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueSchedulerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueSchedulerTest.java
index 9fc4256..f1e3191 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueSchedulerTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/scheduler/queue/QueueSchedulerTest.java
@@ -20,6 +20,7 @@ import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.List;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import org.junit.Test;
import eu.stratosphere.core.io.StringRecord;
@@ -29,9 +30,8 @@ import eu.stratosphere.nephele.executiongraph.ExecutionGraph;
import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
import eu.stratosphere.nephele.executiongraph.GraphConversionException;
import eu.stratosphere.nephele.instance.InstanceManager;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
import eu.stratosphere.nephele.jobgraph.JobInputVertex;
@@ -57,7 +57,7 @@ public class QueueSchedulerTest {
*/
@Override
public void registerInputOutput() {
- new RecordWriter<StringRecord>(this, StringRecord.class);
+ new RecordWriter<StringRecord>(this);
}
/**
@@ -145,7 +145,7 @@ public class QueueSchedulerTest {
final TestDeploymentManager tdm = new TestDeploymentManager();
final QueueScheduler scheduler = new QueueScheduler(tdm, tim);
- final ExecutionGraph executionGraph = createExecutionGraph(ChannelType.INMEMORY, tim);
+ final ExecutionGraph executionGraph = createExecutionGraph(ChannelType.IN_MEMORY, tim);
try {
try {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java
index 4fd1ac1..630f365 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/managementgraph/ManagementGraphTest.java
@@ -19,7 +19,7 @@ import java.util.Iterator;
import org.junit.Test;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.nephele.util.ManagementTestUtils;
@@ -275,7 +275,7 @@ public class ManagementGraphTest {
// Group Edges
new ManagementGroupEdge(groupVertex1, 0, groupVertex2, 0, ChannelType.NETWORK);
new ManagementGroupEdge(groupVertex2, 0, groupVertex3, 0, ChannelType.NETWORK);
- new ManagementGroupEdge(groupVertex3, 0, groupVertex4, 0, ChannelType.INMEMORY);
+ new ManagementGroupEdge(groupVertex3, 0, groupVertex4, 0, ChannelType.IN_MEMORY);
// Edges
new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate1_1, 0, inputGate2_1, 0,
@@ -287,7 +287,7 @@ public class ManagementGraphTest {
new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate2_2, 0, inputGate3_1, 1,
ChannelType.NETWORK);
new ManagementEdge(new ManagementEdgeID(), new ManagementEdgeID(), outputGate3_1, 0, inputGate4_1, 0,
- ChannelType.INMEMORY);
+ ChannelType.IN_MEMORY);
return graph;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/profiling/impl/InstanceProfilerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/profiling/impl/InstanceProfilerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/profiling/impl/InstanceProfilerTest.java
index dda9491..f1b83a6 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/profiling/impl/InstanceProfilerTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/profiling/impl/InstanceProfilerTest.java
@@ -74,7 +74,7 @@ public class InstanceProfilerTest {
@Before
public void setUp() throws Exception {
initMocks(this);
- when(this.infoMock.getAddress()).thenReturn(this.addressMock);
+ when(this.infoMock.address()).thenReturn(this.addressMock);
when(this.addressMock.getHostAddress()).thenReturn("192.168.1.1");
whenNew(FileReader.class).withArguments(InstanceProfiler.PROC_STAT).thenReturn(this.cpuReaderMock);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializerTest.java
deleted file mode 100644
index 1c9efd5..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializerTest.java
+++ /dev/null
@@ -1,358 +0,0 @@
-/***********************************************************************************************************************
- * 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.transferenvelope;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ReadableByteChannel;
-import java.util.ArrayDeque;
-import java.util.Queue;
-
-import org.junit.Test;
-
-import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.BufferFactory;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.MemoryBuffer;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferAvailabilityListener;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProviderBroker;
-import eu.stratosphere.nephele.util.BufferPoolConnector;
-import eu.stratosphere.nephele.util.InterruptibleByteChannel;
-import eu.stratosphere.util.StringUtils;
-
-/**
- * This class contains tests covering the deserialization of a byte stream to a transfer envelope.
- *
- */
-public class DefaultDeserializerTest {
-
- /**
- * The size of the test byte buffers in byte.
- */
- private static final int TEST_BUFFER_CAPACITY = 1024;
-
- /**
- * The sequence number to be used during the tests.
- */
- private static final int SEQUENCE_NUMBER = 0;
-
- /**
- * The job ID to be used during the tests.
- */
- private static final JobID JOB_ID = new JobID();
-
- /**
- * The channel ID to be used during the tests.
- */
- private static final ChannelID CHANNEL_ID = new ChannelID();
-
- /**
- * A dummy implementation of a {@link BufferProvider} which is used in this test.
- * <p>
- * This class is not thread-safe.
- *
- */
- private static final class TestBufferProvider implements BufferProvider {
-
- /**
- * Stores the available byte buffers.
- */
- private final Queue<MemorySegment> bufferPool;
-
- /**
- * Constructs a new test buffer provider.
- *
- * @param numberOfBuffers
- * the number of byte buffers this pool has available.
- */
- private TestBufferProvider(final int numberOfBuffers) {
-
- this.bufferPool = new ArrayDeque<MemorySegment>(numberOfBuffers);
- for (int i = 0; i < numberOfBuffers; ++i) {
- this.bufferPool.add(new MemorySegment(new byte[TEST_BUFFER_CAPACITY]));
- }
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException {
-
- if (this.bufferPool.isEmpty()) {
- return null;
- }
-
- return BufferFactory.createFromMemory(minimumSizeOfBuffer, this.bufferPool.poll(),
- new BufferPoolConnector(this.bufferPool));
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws IOException,
- InterruptedException {
-
- throw new IllegalStateException("requestEmptyBufferBlocking called");
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public int getMaximumBufferSize() {
-
- throw new IllegalStateException("getMaximumBufferSize called");
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public boolean isShared() {
-
- throw new IllegalStateException("isShared called");
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public void reportAsynchronousEvent() {
-
- throw new IllegalStateException("reportAsynchronousEvent called");
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public boolean registerBufferAvailabilityListener(final BufferAvailabilityListener bufferAvailabilityListener) {
-
- throw new IllegalStateException("registerBufferAvailabilityListener called");
- }
- }
-
- /**
- * A dummy implementation of a {@link BufferProviderBroker} which is used during this test.
- * <p>
- * This class is not thread-safe.
- *
- */
- private static final class TestBufferProviderBroker implements BufferProviderBroker {
-
- private final BufferProvider bufferProvider;
-
- private TestBufferProviderBroker(final BufferProvider bufferProvider) {
- this.bufferProvider = bufferProvider;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public BufferProvider getBufferProvider(final JobID jobID, final ChannelID sourceChannelID) throws IOException,
- InterruptedException {
-
- return this.bufferProvider;
- }
- }
-
- /**
- * Constructs an {@link InterruptibleByteChannel} from which the deserializer to be tested can read its data.
- *
- * @param readInterruptPositions
- * the positions after which the byte stream shall be interrupted
- * @param testBufferSize
- * the size of the test buffer to create
- * @return an {@link InterruptibleByteChannel} holding the serialized data in memory
- * @throws IOException
- * thrown if an error occurs while serializing the original data
- */
- private ReadableByteChannel createByteChannel(final int[] readInterruptPositions, final int testBufferSize)
- throws IOException {
-
- final TransferEnvelope te = new TransferEnvelope(SEQUENCE_NUMBER, JOB_ID, CHANNEL_ID);
-
- if (testBufferSize >= 0) {
-
- if (testBufferSize > 100) {
- throw new IllegalStateException("Test buffer size can be 100 bytes at most");
- }
-
- final Queue<MemorySegment> bufferPool = new ArrayDeque<MemorySegment>();
- final MemorySegment ms = new MemorySegment(new byte[TEST_BUFFER_CAPACITY]);
-
- final MemoryBuffer buffer = BufferFactory.createFromMemory(ms.size(), ms, new BufferPoolConnector(bufferPool));
-
- final ByteBuffer srcBuffer = ByteBuffer.allocate(testBufferSize);
- for (int i = 0; i < testBufferSize; ++i) {
- srcBuffer.put((byte) i);
- }
- srcBuffer.flip();
-
- buffer.write(srcBuffer);
- buffer.flip();
- te.setBuffer(buffer);
- }
-
- final DefaultSerializer ds = new DefaultSerializer();
- ds.setTransferEnvelope(te);
-
- final InterruptibleByteChannel ibc = new InterruptibleByteChannel(null, readInterruptPositions);
-
- while (ds.write(ibc));
-
- ibc.switchToReadPhase();
-
- return ibc;
- }
-
- /**
- * Executes the deserialization method.
- *
- * @param rbc
- * the byte channel to read the serialized data from
- * @param bpb
- * the buffer provider broker to request empty buffers from
- * @return the deserialized transfer envelope
- * @throws IOException
- * thrown if an error occurs during the deserialization process
- * @throws NoBufferAvailableException
- * thrown if the buffer provider broker could not provide an empty buffer
- */
- private TransferEnvelope executeDeserialization(final ReadableByteChannel rbc, final BufferProviderBroker bpb)
- throws IOException, NoBufferAvailableException {
-
- final DefaultDeserializer dd = new DefaultDeserializer(bpb);
-
- TransferEnvelope te = dd.getFullyDeserializedTransferEnvelope();
- while (te == null) {
-
- dd.read(rbc);
- te = dd.getFullyDeserializedTransferEnvelope();
- }
-
- assertEquals(SEQUENCE_NUMBER, te.getSequenceNumber());
- assertEquals(JOB_ID, te.getJobID());
- assertEquals(CHANNEL_ID, te.getSource());
-
- return te;
- }
-
- /**
- * Tests the deserialization process of a {@link TransferEnvelope} with a buffer when no interruption of the byte
- * stream.
- */
- @Test
- public void testDeserializationWithBufferAndWithoutInterruption() {
-
- try {
-
- final ReadableByteChannel rbc = createByteChannel(null, 10);
-
- final TestBufferProviderBroker tbpb = new TestBufferProviderBroker(new TestBufferProvider(1));
-
- final TransferEnvelope te = executeDeserialization(rbc, tbpb);
-
- assertNotNull(te.getBuffer());
- assertEquals(10, te.getBuffer().size());
-
- } catch (IOException ioe) {
- fail(StringUtils.stringifyException(ioe));
- } catch (NoBufferAvailableException nbae) {
- fail(StringUtils.stringifyException(nbae));
- }
- }
-
- /**
- * Tests the deserialization process of a {@link TransferEnvelope} with a buffer and interruptions of the byte
- * stream.
- */
- @Test
- public void testDeserializationWithBufferAndInterruptions() {
-
- try {
-
- final ReadableByteChannel rbc = createByteChannel(new int[] { 3, 7, 24, 52 }, 10);
-
- final TestBufferProviderBroker tbpb = new TestBufferProviderBroker(new TestBufferProvider(1));
-
- final TransferEnvelope te = executeDeserialization(rbc, tbpb);
-
- assertNotNull(te.getBuffer());
- assertEquals(10, te.getBuffer().size());
-
- } catch (IOException ioe) {
- fail(StringUtils.stringifyException(ioe));
- } catch (NoBufferAvailableException nbae) {
- fail(StringUtils.stringifyException(nbae));
- }
- }
-
- /**
- * Tests the deserialization process of a {@link TransferEnvelope} without a buffer and without interruptions of the
- * byte stream.
- */
- @Test
- public void testDeserializationWithoutBufferAndInterruptions() {
-
- try {
-
- final ReadableByteChannel rbc = createByteChannel(null, -1);
-
- final TestBufferProviderBroker tbpb = new TestBufferProviderBroker(new TestBufferProvider(1));
-
- final TransferEnvelope te = executeDeserialization(rbc, tbpb);
-
- assertNull(te.getBuffer());
-
- } catch (IOException ioe) {
- fail(StringUtils.stringifyException(ioe));
- } catch (NoBufferAvailableException nbae) {
- fail(StringUtils.stringifyException(nbae));
- }
- }
-
- /**
- * Tests the deserialization process in case the buffer provide cannot deliver an empty buffer to read the byte
- * stream into.
- */
- @Test
- public void testDeserializationWithNoBufferAvailable() {
-
- try {
- final ReadableByteChannel rbc = createByteChannel(null, 10);
- final TestBufferProviderBroker tbpb = new TestBufferProviderBroker(new TestBufferProvider(0));
- executeDeserialization(rbc, tbpb);
-
- } catch (IOException ioe) {
- fail(StringUtils.stringifyException(ioe));
- } catch (NoBufferAvailableException nbae) {
- // Expected exception was successfully caught
- return;
- }
-
- fail("Expected NoBufferAvailableException but has not been thrown");
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializerTest.java
deleted file mode 100644
index a50fbe2..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializerTest.java
+++ /dev/null
@@ -1,313 +0,0 @@
-/***********************************************************************************************************************
- * 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.transferenvelope;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.ArrayDeque;
-import java.util.Deque;
-
-import org.junit.Test;
-
-import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.nephele.io.AbstractID;
-import eu.stratosphere.nephele.io.channels.BufferFactory;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.MemoryBuffer;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.util.BufferPoolConnector;
-import eu.stratosphere.nephele.util.ServerTestUtils;
-
-/**
- * This class contains tests covering the serialization of transfer envelopes to a byte stream.
- *
- */
-public class DefaultSerializerTest {
-
- /**
- * The maximum size of the transfer envelope's buffer.
- */
- private static final int BUFFER_SIZE = 4096; // 4 KB;
-
- /**
- * An arbitrarily chosen byte used to fill the transfer envelope's buffer.
- */
- private static final byte BUFFER_CONTENT = 13;
-
- /**
- * The size of a sequence number.
- */
- private static final int SIZE_OF_SEQ_NR = 4;
-
- /**
- * The size of an ID.
- */
- private static final int SIZE_OF_ID = 16;
-
- /**
- * The size of an integer number.
- */
- private static final int SIZE_OF_INTEGER = 4;
-
- /**
- * The job ID used during the serialization process.
- */
- private final JobID jobID = new JobID();
-
- /**
- * The target channel ID used during the serialization process.
- */
- private final ChannelID sourceChannelID = new ChannelID();
-
- /**
- * Auxiliary class to explicitly access the internal buffer of an ID object.
- *
- */
- private static class SerializationTestID extends AbstractID {
-
- /**
- * Constructs a new ID.
- *
- * @param content
- * a byte buffer representing the ID
- */
- private SerializationTestID(byte[] content) {
- super(content);
- }
- }
-
- /**
- * This test checks the correctness of the serialization of {@link TransferEnvelope} objects.
- */
- @Test
- public void testSerialization() {
-
- try {
-
- // Generate test file
- final File testFile = generateDataStream();
-
- // Analyze the test file
- analyzeStream(testFile);
-
- // Delete the test file
- testFile.delete();
-
- } catch (IOException e) {
- fail(e.getMessage());
- }
- }
-
- /**
- * Generates and serializes a series of {@link TransferEnvelope} objects to a random file.
- *
- * @return the file containing the serializes envelopes
- * @throws IOException
- * thrown if an I/O error occurs while writing the envelopes
- */
- private File generateDataStream() throws IOException {
-
- final File outputFile = new File(ServerTestUtils.getTempDir() + File.separator
- + ServerTestUtils.getRandomFilename());
- final FileOutputStream outputStream = new FileOutputStream(outputFile);
- final FileChannel fileChannel = outputStream.getChannel();
- final Deque<MemorySegment> recycleQueue = new ArrayDeque<MemorySegment>();
- final DefaultSerializer serializer = new DefaultSerializer();
- final MemorySegment byteBuffer = new MemorySegment(new byte[BUFFER_SIZE]);
- final ByteBuffer initBuffer = ByteBuffer.allocate(1);
-
- // The byte buffer is initialized from this buffer
- initBuffer.put(BUFFER_CONTENT);
- initBuffer.flip();
-
- // Put byte buffer to recycled queue
- recycleQueue.add(byteBuffer);
-
- for (int i = 0; i < BUFFER_SIZE; i++) {
-
- final MemoryBuffer buffer = BufferFactory.createFromMemory(i, recycleQueue.poll(), new BufferPoolConnector(
- recycleQueue));
-
- // Initialize buffer
- for (int j = 0; j < i; j++) {
- buffer.write(initBuffer);
- initBuffer.position(0);
- }
- buffer.flip();
-
- final TransferEnvelope transferEnvelope = new TransferEnvelope(i, this.jobID, this.sourceChannelID);
- transferEnvelope.setBuffer(buffer);
-
- // set envelope to be serialized and write it to file channel
- serializer.setTransferEnvelope(transferEnvelope);
- while (serializer.write(fileChannel));
-
- // Put buffer back to the recycling queue
- buffer.recycleBuffer();
- }
-
- fileChannel.close();
-
- return outputFile;
- }
-
- /**
- * Analyzes the given test file and checks whether its content matches Nephele's serialization pattern.
- *
- * @param testFile
- * the test file to analyze
- * @throws IOException
- * thrown if an I/O error occurs while reading the test file
- */
- private void analyzeStream(File testFile) throws IOException {
-
- FileInputStream fileInputStream = new FileInputStream(testFile);
-
- for (int i = 0; i < BUFFER_SIZE; i++) {
-
- readAndCheckSequenceNumber(fileInputStream, i);
- readAndCheckID(fileInputStream, this.jobID);
- readAndCheckID(fileInputStream, this.sourceChannelID);
- readAndCheckNotificationList(fileInputStream);
- readAndCheckBuffer(fileInputStream, i);
- }
-
- fileInputStream.close();
- }
-
- /**
- * Attempts to read a buffer of the given size from the file stream and checks the buffer's content.
- *
- * @param fileInputStream
- * the file stream to read from
- * @param expectedBufferSize
- * the expected size of the buffer
- * @throws IOException
- * thrown if an error occurs while reading from the file stream
- */
- private static void readAndCheckBuffer(FileInputStream fileInputStream, int expectedBufferSize) throws IOException {
-
- // Check if buffer exists
- assertEquals(1L, fileInputStream.read());
-
- byte[] temp = new byte[SIZE_OF_INTEGER];
- fileInputStream.read(temp);
- int bufferSize = bufferToInteger(temp);
-
- assertEquals(expectedBufferSize, bufferSize);
-
- byte[] buffer = new byte[bufferSize];
- int r = fileInputStream.read(buffer);
- for (int i = 0; i < buffer.length; i++) {
- assertEquals(BUFFER_CONTENT, buffer[i]);
- }
- }
-
- /**
- * Attempts to read an empty notification list from the given file input stream.
- *
- * @param fileInputStream
- * the file input stream to read from
- * @throws IOException
- * thrown if an I/O occurs while reading data from the stream
- */
- private void readAndCheckNotificationList(FileInputStream fileInputStream) throws IOException {
-
- if (fileInputStream.read() != 0) {
-
- byte[] temp = new byte[SIZE_OF_INTEGER];
-
- fileInputStream.read(temp);
- final int sizeOfDataBlock = bufferToInteger(temp);
-
- assertEquals(SIZE_OF_INTEGER, sizeOfDataBlock);
-
- fileInputStream.read(temp);
- final int sizeOfNotificationList = bufferToInteger(temp);
-
- assertEquals(0, sizeOfNotificationList);
- }
- }
-
- /**
- * Attempts to read an integer number from the given file input stream and compares it to
- * <code>expectedSequenceNumber</code>.
- *
- * @param fileInputStream
- * the file input stream to read from
- * @param expectedSeqNumber
- * the integer number the read number is expected to match
- * @throws IOException
- * thrown if an I/O occurs while reading data from the stream
- */
- private void readAndCheckSequenceNumber(FileInputStream fileInputStream, int expectedSeqNumber) throws IOException {
-
- byte[] temp = new byte[SIZE_OF_SEQ_NR];
- fileInputStream.read(temp);
- int seqNumber = bufferToInteger(temp);
-
- assertEquals(seqNumber, expectedSeqNumber);
- }
-
- /**
- * Attempts to read a channel ID from the given file input stream and compares it to <code>expectedChannelID</code>.
- *
- * @param fileInputStream
- * the file input stream to read from
- * @param expectedID
- * the ID which the read ID is expected to match
- * @throws IOException
- * thrown if an I/O occurs while reading data from the stream
- */
- private void readAndCheckID(FileInputStream fileInputStream, AbstractID expectedID) throws IOException {
-
- byte[] temp = new byte[SIZE_OF_INTEGER];
- fileInputStream.read(temp);
-
- final int sizeOfID = bufferToInteger(temp); // ID has fixed size and therefore does not announce its size
-
- assertEquals(sizeOfID, SIZE_OF_ID);
-
- byte[] id = new byte[sizeOfID];
- fileInputStream.read(id);
-
- final AbstractID channelID = new SerializationTestID(id);
- assertEquals(expectedID, channelID);
- }
-
- /**
- * Converts the first four bytes of the provided buffer's content to an integer number.
- *
- * @param buffer
- * the buffer to convert
- * @return the integer number converted from the first four bytes of the buffer's content
- */
- private static int bufferToInteger(byte[] buffer) {
-
- int integer = 0;
-
- for (int i = 0; i < SIZE_OF_INTEGER; ++i) {
- integer |= (buffer[SIZE_OF_INTEGER - 1 - i] & 0xff) << (i << 3);
- }
-
- return integer;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/BufferPoolConnector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/BufferPoolConnector.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/BufferPoolConnector.java
deleted file mode 100644
index 3f2c79e..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/BufferPoolConnector.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/***********************************************************************************************************************
- * 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.util;
-
-import java.util.Queue;
-
-import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.nephele.io.channels.MemoryBufferPoolConnector;
-
-/**
- * This is a simple implementation of a {@link MemoryBufferPoolConnector} used for the server unit tests.
- * <p>
- * This class is thread-safe.
- *
- */
-public final class BufferPoolConnector implements MemoryBufferPoolConnector {
-
- /**
- * Reference to the memory pool the byte buffer was originally taken from.
- */
- private final Queue<MemorySegment> memoryPool;
-
- /**
- * Constructs a new buffer pool connector
- *
- * @param bufferPool
- * a reference to the memory pool the byte buffer was originally taken from
- */
- public BufferPoolConnector(final Queue<MemorySegment> bufferPool) {
- this.memoryPool = bufferPool;
- }
-
-
- @Override
- public void recycle(final MemorySegment memSeg) {
-
- synchronized (this.memoryPool) {
- this.memoryPool.add(memSeg);
- this.memoryPool.notify();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/DiscardingRecycler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/DiscardingRecycler.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/DiscardingRecycler.java
new file mode 100644
index 0000000..9d4d2a5
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/DiscardingRecycler.java
@@ -0,0 +1,24 @@
+/***********************************************************************************************************************
+ *
+ * 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.util;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.BufferRecycler;
+
+public class DiscardingRecycler implements BufferRecycler {
+
+ @Override
+ public void recycle(MemorySegment memSeg) {}
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineReader.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineReader.java
new file mode 100644
index 0000000..fcb4fa1
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineReader.java
@@ -0,0 +1,80 @@
+/***********************************************************************************************************************
+ * 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.util;
+
+import java.util.Iterator;
+
+import eu.stratosphere.core.fs.FSDataInputStream;
+import eu.stratosphere.core.fs.FileInputSplit;
+import eu.stratosphere.core.fs.FileSystem;
+import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.runtime.io.api.RecordWriter;
+import eu.stratosphere.nephele.template.AbstractFileInputTask;
+import eu.stratosphere.runtime.fs.LineReader;
+
+/**
+ * A file line reader reads the associated file input splits line by line and outputs the lines as string records.
+ *
+ */
+public class FileLineReader extends AbstractFileInputTask {
+
+ private RecordWriter<StringRecord> output = null;
+
+ @Override
+ public void invoke() throws Exception {
+
+ output.initializeSerializers();
+
+ final Iterator<FileInputSplit> splitIterator = getFileInputSplits();
+
+ while (splitIterator.hasNext()) {
+
+ final FileInputSplit split = splitIterator.next();
+
+ long start = split.getStart();
+ long length = split.getLength();
+
+ final FileSystem fs = FileSystem.get(split.getPath().toUri());
+
+ final FSDataInputStream fdis = fs.open(split.getPath());
+
+ final LineReader lineReader = new LineReader(fdis, start, length, (1024 * 1024));
+
+ byte[] line = lineReader.readLine();
+
+ while (line != null) {
+
+ // Create a string object from the data read
+ StringRecord str = new StringRecord();
+ str.set(line);
+
+ // Send out string
+ output.emit(str);
+
+ line = lineReader.readLine();
+ }
+
+ // Close the stream;
+ lineReader.close();
+ }
+
+ this.output.flush();
+ }
+
+ @Override
+ public void registerInputOutput() {
+ output = new RecordWriter<StringRecord>(this);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineWriter.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineWriter.java
new file mode 100644
index 0000000..bc738df
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/FileLineWriter.java
@@ -0,0 +1,75 @@
+/***********************************************************************************************************************
+ * 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.util;
+
+import eu.stratosphere.core.fs.FSDataOutputStream;
+import eu.stratosphere.core.fs.FileStatus;
+import eu.stratosphere.core.fs.FileSystem;
+import eu.stratosphere.core.fs.Path;
+import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.nephele.template.AbstractFileOutputTask;
+
+/**
+ * A file line writer reads string records its input gate and writes them to the associated output file.
+ *
+ */
+public class FileLineWriter extends AbstractFileOutputTask {
+
+ /**
+ * The record reader through which incoming string records are received.
+ */
+ private RecordReader<StringRecord> input = null;
+
+
+ @Override
+ public void invoke() throws Exception {
+
+ Path outputPath = getFileOutputPath();
+
+ FileSystem fs = FileSystem.get(outputPath.toUri());
+ if (fs.exists(outputPath)) {
+ FileStatus status = fs.getFileStatus(outputPath);
+
+ if (status.isDir()) {
+ outputPath = new Path(outputPath.toUri().toString() + "/file_" + getIndexInSubtaskGroup() + ".txt");
+ }
+ }
+
+ final FSDataOutputStream outputStream = fs.create(outputPath, true);
+
+ while (this.input.hasNext()) {
+
+ StringRecord record = this.input.next();
+ byte[] recordByte = (record.toString() + "\r\n").getBytes();
+ outputStream.write(recordByte, 0, recordByte.length);
+ }
+
+ outputStream.close();
+
+ }
+
+
+ @Override
+ public void registerInputOutput() {
+ this.input = new RecordReader<StringRecord>(this, StringRecord.class);
+ }
+
+
+ @Override
+ public int getMaximumNumberOfSubtasks() {
+ // The default implementation always returns -1
+ return -1;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java
new file mode 100644
index 0000000..09b244f
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java
@@ -0,0 +1,76 @@
+/***********************************************************************************************************************
+ *
+ * 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.util;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.BufferRecycler;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
+
+import java.io.IOException;
+import java.util.Random;
+
+public class TestBufferProvider implements BufferProvider {
+
+ private final BufferRecycler recycler = new DiscardingRecycler();
+
+ private final Random rnd = new Random();
+
+ private final int sizeOfMemorySegments;
+
+ private final float probabilityForNoneAvailable;
+
+
+ public TestBufferProvider(int sizeOfMemorySegments) {
+ this(sizeOfMemorySegments, -1.0f);
+ }
+
+ public TestBufferProvider(int sizeOfMemorySegments, float probabilityForNoneAvailable) {
+ this.sizeOfMemorySegments = sizeOfMemorySegments;
+ this.probabilityForNoneAvailable = probabilityForNoneAvailable;
+ }
+
+ @Override
+ public Buffer requestBuffer(int sizeOfBuffer) throws IOException {
+ if (rnd.nextFloat() < this.probabilityForNoneAvailable) {
+ return null;
+ } else {
+ MemorySegment segment = new MemorySegment(new byte[this.sizeOfMemorySegments]);
+ return new Buffer(segment, sizeOfBuffer, this.recycler);
+ }
+ }
+
+ @Override
+ public Buffer requestBufferBlocking(int sizeOfBuffer) throws IOException, InterruptedException {
+ MemorySegment segment = new MemorySegment(new byte[this.sizeOfMemorySegments]);
+ return new Buffer(segment, sizeOfBuffer, this.recycler);
+ }
+
+ @Override
+ public int getBufferSize() {
+ return Integer.MAX_VALUE;
+ }
+
+ @Override
+ public void reportAsynchronousEvent() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean registerBufferAvailabilityListener(BufferAvailabilityListener bufferAvailabilityListener) {
+ throw new UnsupportedOperationException();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSinkTaskTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSinkTaskTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSinkTaskTest.java
index 6c66b78..bfd0d42 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSinkTaskTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSinkTaskTest.java
@@ -42,8 +42,12 @@ import eu.stratosphere.types.IntValue;
import eu.stratosphere.types.Key;
import eu.stratosphere.types.Record;
-public class DataSinkTaskTest extends TaskTestBase
-{
+public class DataSinkTaskTest extends TaskTestBase {
+
+ private static final int MEMORY_MANAGER_SIZE = 1024 * 1024;
+
+ private static final int NETWORK_BUFFER_SIZE = 1024;
+
private static final Log LOG = LogFactory.getLog(DataSinkTaskTest.class);
private final String tempTestPath = Path.constructTestPath("dst_test");
@@ -61,8 +65,8 @@ public class DataSinkTaskTest extends TaskTestBase
int keyCnt = 100;
int valCnt = 20;
-
- super.initEnvironment(1024 * 1024);
+
+ super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
super.addInput(new UniformRecordGenerator(keyCnt, valCnt, false), 0);
DataSinkTask<Record> testTask = new DataSinkTask<Record>();
@@ -127,8 +131,8 @@ public class DataSinkTaskTest extends TaskTestBase
int keyCnt = 100;
int valCnt = 20;
-
- super.initEnvironment(1024 * 1024);
+
+ super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
super.addInput(new UniformRecordGenerator(keyCnt, valCnt, 0, 0, false), 0);
super.addInput(new UniformRecordGenerator(keyCnt, valCnt, keyCnt, 0, false), 0);
super.addInput(new UniformRecordGenerator(keyCnt, valCnt, keyCnt*2, 0, false), 0);
@@ -197,8 +201,8 @@ public class DataSinkTaskTest extends TaskTestBase
int keyCnt = 100;
int valCnt = 20;
-
- super.initEnvironment(1024 * 1024 * 4);
+
+ super.initEnvironment(MEMORY_MANAGER_SIZE * 4, NETWORK_BUFFER_SIZE);
super.addInput(new UniformRecordGenerator(keyCnt, valCnt, true), 0);
DataSinkTask<Record> testTask = new DataSinkTask<Record>();
@@ -275,8 +279,8 @@ public class DataSinkTaskTest extends TaskTestBase
int keyCnt = 100;
int valCnt = 20;
-
- super.initEnvironment(1024 * 1024);
+
+ super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
super.addInput(new UniformRecordGenerator(keyCnt, valCnt, false), 0);
DataSinkTask<Record> testTask = new DataSinkTask<Record>();
@@ -307,8 +311,8 @@ public class DataSinkTaskTest extends TaskTestBase
int keyCnt = 100;
int valCnt = 20;
-
- super.initEnvironment(4 * 1024 * 1024);
+
+ super.initEnvironment(MEMORY_MANAGER_SIZE * 4, NETWORK_BUFFER_SIZE);
super.addInput(new UniformRecordGenerator(keyCnt, valCnt, true), 0);
DataSinkTask<Record> testTask = new DataSinkTask<Record>();
@@ -343,8 +347,8 @@ public class DataSinkTaskTest extends TaskTestBase
@Test
public void testCancelDataSinkTask() {
-
- super.initEnvironment(1024 * 1024);
+
+ super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
super.addInput(new InfiniteInputIterator(), 0);
final DataSinkTask<Record> testTask = new DataSinkTask<Record>();
@@ -385,8 +389,8 @@ public class DataSinkTaskTest extends TaskTestBase
@Test
@SuppressWarnings("unchecked")
public void testCancelSortingDataSinkTask() {
-
- super.initEnvironment(4 * 1024 * 1024);
+
+ super.initEnvironment(MEMORY_MANAGER_SIZE * 4, NETWORK_BUFFER_SIZE);
super.addInput(new InfiniteInputIterator(), 0);
final DataSinkTask<Record> testTask = new DataSinkTask<Record>();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSourceTaskTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSourceTaskTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSourceTaskTest.java
index 0198db2..732cf8d 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSourceTaskTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/DataSourceTaskTest.java
@@ -37,8 +37,12 @@ import eu.stratosphere.types.IntValue;
import eu.stratosphere.types.Record;
import eu.stratosphere.util.MutableObjectIterator;
-public class DataSourceTaskTest extends TaskTestBase
-{
+public class DataSourceTaskTest extends TaskTestBase {
+
+ private static final int MEMORY_MANAGER_SIZE = 1024 * 1024;
+
+ private static final int NETWORK_BUFFER_SIZE = 1024;
+
private List<Record> outList;
private String tempTestPath = Path.constructTestPath("dst_test");
@@ -50,11 +54,9 @@ public class DataSourceTaskTest extends TaskTestBase
tempTestFile.delete();
}
}
-
@Test
public void testDataSourceTask() {
-
int keyCnt = 100;
int valCnt = 20;
@@ -67,7 +69,7 @@ public class DataSourceTaskTest extends TaskTestBase
Assert.fail("Unable to set-up test input file");
}
- super.initEnvironment(1024 * 1024);
+ super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
super.addOutput(this.outList);
DataSourceTask<Record> testTask = new DataSourceTask<Record>();
@@ -110,7 +112,6 @@ public class DataSourceTaskTest extends TaskTestBase
@Test
public void testFailingDataSourceTask() {
-
int keyCnt = 20;
int valCnt = 10;
@@ -122,8 +123,8 @@ public class DataSourceTaskTest extends TaskTestBase
} catch (IOException e1) {
Assert.fail("Unable to set-up test input file");
}
-
- super.initEnvironment(1024 * 1024);
+
+ super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
super.addOutput(this.outList);
DataSourceTask<Record> testTask = new DataSourceTask<Record>();
@@ -148,11 +149,10 @@ public class DataSourceTaskTest extends TaskTestBase
@Test
public void testCancelDataSourceTask() {
-
int keyCnt = 20;
int valCnt = 4;
-
- super.initEnvironment(1024 * 1024);
+
+ super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
super.addOutput(new NirvanaOutputList());
try {
@@ -184,7 +184,7 @@ public class DataSourceTaskTest extends TaskTestBase
try {
tct.join();
- taskRunner.join();
+ taskRunner.join();
} catch(InterruptedException ie) {
Assert.fail("Joining threads failed");
}
@@ -192,12 +192,10 @@ public class DataSourceTaskTest extends TaskTestBase
// assert that temp file was created
File tempTestFile = new File(this.tempTestPath);
Assert.assertTrue("Temp output file does not exist",tempTestFile.exists());
-
}
- private static class InputFilePreparator
- {
+ private static class InputFilePreparator {
public static void prepareInputFile(MutableObjectIterator<Record> inIt, String inputFilePath, boolean insertInvalidData)
throws IOException
{
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/chaining/ChainTaskTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/chaining/ChainTaskTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/chaining/ChainTaskTest.java
index 100bf7b..dda215e 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/chaining/ChainTaskTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/chaining/ChainTaskTest.java
@@ -43,7 +43,11 @@ import eu.stratosphere.util.LogUtils;
public class ChainTaskTest extends TaskTestBase {
-
+
+ private static final int MEMORY_MANAGER_SIZE = 1024 * 1024 * 3;
+
+ private static final int NETWORK_BUFFER_SIZE = 1024;
+
private final List<Record> outList = new ArrayList<Record>();
@SuppressWarnings("unchecked")
@@ -65,9 +69,8 @@ public class ChainTaskTest extends TaskTestBase {
final int valCnt = 20;
try {
-
// environment
- initEnvironment(3*1024*1024);
+ super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
addInput(new UniformRecordGenerator(keyCnt, valCnt, false), 0);
addOutput(this.outList);
@@ -123,7 +126,7 @@ public class ChainTaskTest extends TaskTestBase {
try {
// environment
- initEnvironment(3*1024*1024);
+ super.initEnvironment(MEMORY_MANAGER_SIZE, NETWORK_BUFFER_SIZE);
addInput(new UniformRecordGenerator(keyCnt, valCnt, false), 0);
addOutput(this.outList);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
index cc00387..c66d821 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
@@ -26,6 +26,7 @@ import org.apache.commons.lang.NotImplementedException;
import org.junit.Test;
import eu.stratosphere.api.common.typeutils.TypeComparator;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
import eu.stratosphere.api.common.typeutils.base.IntSerializer;
import eu.stratosphere.api.java.typeutils.runtime.record.RecordComparatorFactory;
import eu.stratosphere.api.java.typeutils.runtime.record.RecordSerializerFactory;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
index 8afc78f..0b968d8 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
@@ -27,6 +27,8 @@ import org.junit.Test;
import eu.stratosphere.api.common.distributions.DataDistribution;
import eu.stratosphere.api.common.distributions.UniformIntegerDistribution;
+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.pact.runtime.shipping.RecordOutputEmitter;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
index 8e25082..a397312 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
@@ -13,38 +13,40 @@
package eu.stratosphere.pact.runtime.test.util;
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.FutureTask;
-
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.core.fs.Path;
import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.core.memory.MemorySegment;
import eu.stratosphere.nephele.execution.Environment;
-import eu.stratosphere.nephele.io.ChannelSelector;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.InputChannelResult;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.MutableRecordDeserializerFactory;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.RecordAvailabilityListener;
-import eu.stratosphere.nephele.io.RecordDeserializerFactory;
-import eu.stratosphere.nephele.io.RuntimeInputGate;
-import eu.stratosphere.nephele.io.RuntimeOutputGate;
-import eu.stratosphere.nephele.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.gates.InputChannelResult;
+import eu.stratosphere.runtime.io.gates.RecordAvailabilityListener;
+import eu.stratosphere.runtime.io.serialization.AdaptiveSpanningRecordDeserializer;
+import eu.stratosphere.runtime.io.Buffer;
+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.JobID;
import eu.stratosphere.nephele.protocols.AccumulatorProtocol;
import eu.stratosphere.nephele.services.iomanager.IOManager;
import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
import eu.stratosphere.nephele.services.memorymanager.spi.DefaultMemoryManager;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
+import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool;
+import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPoolOwner;
import eu.stratosphere.nephele.template.InputSplitProvider;
+import eu.stratosphere.runtime.io.serialization.RecordDeserializer;
+import eu.stratosphere.runtime.io.serialization.RecordDeserializer.DeserializationResult;
import eu.stratosphere.types.Record;
import eu.stratosphere.util.MutableObjectIterator;
-public class MockEnvironment implements Environment {
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+public class MockEnvironment implements Environment, BufferProvider, LocalBufferPoolOwner {
private final MemoryManager memManager;
@@ -56,21 +58,24 @@ public class MockEnvironment implements Environment {
private final Configuration taskConfiguration;
- private final List<RuntimeInputGate<Record>> inputs;
+ private final List<InputGate<Record>> inputs;
- private final List<RuntimeOutputGate<Record>> outputs;
+ private final List<OutputGate> outputs;
private final JobID jobID = new JobID();
- public MockEnvironment(long memorySize, MockInputSplitProvider inputSplitProvider) {
+ private final Buffer mockBuffer;
+
+ public MockEnvironment(long memorySize, MockInputSplitProvider inputSplitProvider, int bufferSize) {
this.jobConfiguration = new Configuration();
this.taskConfiguration = new Configuration();
- this.inputs = new LinkedList<RuntimeInputGate<Record>>();
- this.outputs = new LinkedList<RuntimeOutputGate<Record>>();
+ this.inputs = new LinkedList<InputGate<Record>>();
+ this.outputs = new LinkedList<OutputGate>();
this.memManager = new DefaultMemoryManager(memorySize);
this.ioManager = new IOManager(System.getProperty("java.io.tmpdir"));
this.inputSplitProvider = inputSplitProvider;
+ this.mockBuffer = new Buffer(new MemorySegment(new byte[bufferSize]), bufferSize, null);
}
public void addInput(MutableObjectIterator<Record> inputIterator) {
@@ -103,13 +108,62 @@ public class MockEnvironment implements Environment {
return this.jobID;
}
+ @Override
+ public Buffer requestBuffer(int minBufferSize) throws IOException {
+ return mockBuffer;
+ }
+
+ @Override
+ public Buffer requestBufferBlocking(int minBufferSize) throws IOException, InterruptedException {
+ return mockBuffer;
+ }
+
+ @Override
+ public int getBufferSize() {
+ return this.mockBuffer.size();
+ }
+
+ @Override
+ public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+ return false;
+ }
+
+ @Override
+ public int getNumberOfChannels() {
+ return 1;
+ }
+
+ @Override
+ public void setDesignatedNumberOfBuffers(int numBuffers) {
+
+ }
+
+ @Override
+ public void clearLocalBufferPool() {
+
+ }
+
+ @Override
+ public void registerGlobalBufferPool(GlobalBufferPool globalBufferPool) {
+
+ }
+
+ @Override
+ public void logBufferUtilization() {
+
+ }
+
+ @Override
+ public void reportAsynchronousEvent() {
- private static class MockInputGate extends RuntimeInputGate<Record> {
+ }
+
+ private static class MockInputGate extends InputGate<Record> {
private MutableObjectIterator<Record> it;
public MockInputGate(int id, MutableObjectIterator<Record> it) {
- super(new JobID(), new GateID(), MutableRecordDeserializerFactory.<Record>get(), id);
+ super(new JobID(), new GateID(), id);
this.it = it;
}
@@ -132,18 +186,43 @@ public class MockEnvironment implements Environment {
}
}
- private static class MockOutputGate extends RuntimeOutputGate<Record> {
+ private class MockOutputGate extends OutputGate {
private List<Record> out;
+ private RecordDeserializer<Record> deserializer;
+
+ private Record record;
+
public MockOutputGate(int index, List<Record> outList) {
- super(new JobID(), new GateID(), Record.class, index, null, false);
+ super(new JobID(), new GateID(), index);
this.out = outList;
+ this.deserializer = new AdaptiveSpanningRecordDeserializer<Record>();
+ this.record = new Record();
}
@Override
- public void writeRecord(Record record) throws IOException, InterruptedException {
- out.add(record.createCopy());
+ public void sendBuffer(Buffer buffer, int targetChannel) throws IOException, InterruptedException {
+
+ this.deserializer.setNextMemorySegment(MockEnvironment.this.mockBuffer.getMemorySegment(), MockEnvironment.this.mockBuffer.size());
+
+ while (this.deserializer.hasUnfinishedData()) {
+ DeserializationResult result = this.deserializer.getNextRecord(this.record);
+
+ if (result.isFullRecord()) {
+ this.out.add(this.record.createCopy());
+ }
+
+ if (result == DeserializationResult.LAST_RECORD_FROM_BUFFER ||
+ result == DeserializationResult.PARTIAL_RECORD) {
+ break;
+ }
+ }
+ }
+
+ @Override
+ public int getNumChannels() {
+ return 1;
}
}
@@ -188,11 +267,6 @@ public class MockEnvironment implements Environment {
}
@Override
- public GateID getNextUnboundOutputGateID() {
- return null;
- }
-
- @Override
public int getNumberOfOutputGates() {
return this.outputs.size();
}
@@ -203,16 +277,6 @@ public class MockEnvironment implements Environment {
}
@Override
- public void registerOutputGate(final OutputGate<? extends IOReadableWritable> outputGate) {
- // Nothing to do here
- }
-
- @Override
- public void registerInputGate(final InputGate<? extends IOReadableWritable> inputGate) {
- // Nothing to do here
- }
-
- @Override
public Set<ChannelID> getOutputChannelIDs() {
throw new IllegalStateException("getOutputChannelIDs called on MockEnvironment");
}
@@ -242,18 +306,14 @@ public class MockEnvironment implements Environment {
throw new IllegalStateException("getInputChannelIDsOfGate called on MockEnvironment");
}
- @SuppressWarnings("unchecked")
@Override
- public <T extends IOReadableWritable> OutputGate<T> createOutputGate(GateID gateID, Class<T> outputClass,
- ChannelSelector<T> selector, boolean isBroadcast)
+ public OutputGate createAndRegisterOutputGate()
{
- return (OutputGate<T>) this.outputs.remove(0);
+ return this.outputs.remove(0);
}
- @SuppressWarnings("unchecked")
@Override
- public <T extends IOReadableWritable> InputGate<T> createInputGate(GateID gateID,
- RecordDeserializerFactory<T> deserializerFactory)
+ public <T extends IOReadableWritable> InputGate<T> createAndRegisterInputGate()
{
return (InputGate<T>) this.inputs.remove(0);
}
@@ -275,8 +335,7 @@ public class MockEnvironment implements Environment {
}
@Override
- public Map<String, FutureTask<Path>> getCopyTask() {
- return null;
+ public BufferProvider getOutputBufferProvider() {
+ return this;
}
-
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java
index 826113c..a60b479 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/TaskTestBase.java
@@ -49,10 +49,10 @@ public abstract class TaskTestBase {
protected MockEnvironment mockEnv;
- public void initEnvironment(long memorySize) {
+ public void initEnvironment(long memorySize, int bufferSize) {
this.memorySize = memorySize;
this.inputSplitProvider = new MockInputSplitProvider();
- this.mockEnv = new MockEnvironment(this.memorySize, this.inputSplitProvider);
+ this.mockEnv = new MockEnvironment(this.memorySize, this.inputSplitProvider, bufferSize);
}
public void addInput(MutableObjectIterator<Record> input, int groupId) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/LineReaderTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/LineReaderTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/LineReaderTest.java
new file mode 100644
index 0000000..af46689
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/LineReaderTest.java
@@ -0,0 +1,78 @@
+/***********************************************************************************************************************
+ * 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.fs;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.PrintWriter;
+
+import org.junit.Test;
+
+import eu.stratosphere.core.fs.FSDataInputStream;
+import eu.stratosphere.core.fs.Path;
+import eu.stratosphere.core.fs.local.LocalFileSystem;
+import eu.stratosphere.nephele.util.CommonTestUtils;
+
+/**
+ * This class tests the functionality of the LineReader class using a local filesystem.
+ *
+ */
+
+public class LineReaderTest {
+
+ /**
+ * This test tests the LineReader. So far only under usual conditions.
+ */
+ @Test
+ public void testLineReader() {
+ final File testfile = new File(CommonTestUtils.getTempDir() + File.separator
+ + CommonTestUtils.getRandomFilename());
+ final Path pathtotestfile = new Path(testfile.toURI().getPath());
+
+ try {
+ PrintWriter pw = new PrintWriter(testfile, "UTF8");
+
+ for (int i = 0; i < 100; i++) {
+ pw.append("line\n");
+ }
+ pw.close();
+
+ LocalFileSystem lfs = new LocalFileSystem();
+ FSDataInputStream fis = lfs.open(pathtotestfile);
+
+ // first, we test under "usual" conditions
+ final LineReader lr = new LineReader(fis, 0, testfile.length(), 256);
+
+ byte[] buffer;
+ int linecount = 0;
+ while ((buffer = lr.readLine()) != null) {
+ assertEquals(new String(buffer, "UTF8"), "line");
+ linecount++;
+ }
+ assertEquals(linecount, 100);
+
+ // the linereader can not handle situations with larger length than the total file...
+
+ } catch (Exception e) {
+ fail(e.toString());
+ e.printStackTrace();
+ } finally {
+ testfile.delete();
+ }
+
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/s3/S3FileSystemTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/s3/S3FileSystemTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/s3/S3FileSystemTest.java
new file mode 100644
index 0000000..30b5219
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/fs/s3/S3FileSystemTest.java
@@ -0,0 +1,460 @@
+/***********************************************************************************************************************
+ * 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.fs.s3;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import eu.stratosphere.configuration.Configuration;
+import eu.stratosphere.configuration.GlobalConfiguration;
+import eu.stratosphere.core.fs.BlockLocation;
+import eu.stratosphere.core.fs.FSDataInputStream;
+import eu.stratosphere.core.fs.FSDataOutputStream;
+import eu.stratosphere.core.fs.FileStatus;
+import eu.stratosphere.core.fs.FileSystem;
+import eu.stratosphere.core.fs.Path;
+
+/**
+ * This test checks the S3 implementation of the {@link FileSystem} interface.
+ *
+ */
+public class S3FileSystemTest {
+
+ /**
+ * The length of the bucket/object names used in this test.
+ */
+ private static final int NAME_LENGTH = 32;
+
+ /**
+ * The alphabet to generate the random bucket/object names from.
+ */
+ private static final char[] ALPHABET = { 'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o',
+ 'p', 'q', 'r', 's', 't', 'u', 'v', 'w', 'x', 'y', 'z', '0', '1', '2', '3', '4', '5', '6', '7', '8', '9' };
+
+ /**
+ * The size of the byte buffer used during the tests in bytes.
+ */
+ private static final int TEST_BUFFER_SIZE = 128;
+
+ /**
+ * The size of the small test file in bytes.
+ */
+ private static final int SMALL_FILE_SIZE = 512;
+
+ /**
+ * The size of the large test file in bytes.
+ */
+ private static final int LARGE_FILE_SIZE = 1024 * 1024 * 12; // 12 MB
+
+ /**
+ * The modulus to be used when generating the test data. Must not be larger than 128.
+ */
+ private static final int MODULUS = 128;
+
+ private static final String S3_BASE_URI = "s3:///";
+
+ /**
+ * Tries to read the AWS access key and the AWS secret key from the environments variables. If accessing these keys
+ * fails, all tests will be skipped and marked as successful.
+ */
+ @Before
+ public void initKeys() {
+ final String accessKey = System.getenv("AK");
+ final String secretKey = System.getenv("SK");
+
+ if (accessKey != null || secretKey != null) {
+ Configuration conf = new Configuration();
+ if (accessKey != null) {
+ conf.setString(S3FileSystem.S3_ACCESS_KEY_KEY, accessKey);
+ }
+ if (secretKey != null) {
+ conf.setString(S3FileSystem.S3_SECRET_KEY_KEY, secretKey);
+ }
+ GlobalConfiguration.includeConfiguration(conf);
+ }
+ }
+
+ /**
+ * This test creates and deletes a bucket inside S3 and checks it is correctly displayed inside the directory
+ * listing.
+ */
+ @Test
+ public void createAndDeleteBucketTest() {
+
+ if (!testActivated()) {
+ return;
+ }
+
+ final String bucketName = getRandomName();
+ final Path bucketPath = new Path(S3_BASE_URI + bucketName + Path.SEPARATOR);
+
+ try {
+
+ final FileSystem fs = bucketPath.getFileSystem();
+
+ // Create directory
+ fs.mkdirs(bucketPath);
+
+ // Check if directory is correctly displayed in file system hierarchy
+ final FileStatus[] content = fs.listStatus(new Path(S3_BASE_URI));
+ boolean entryFound = false;
+ for (final FileStatus entry : content) {
+ if (bucketPath.equals(entry.getPath())) {
+ entryFound = true;
+ break;
+ }
+ }
+
+ if (!entryFound) {
+ fail("Cannot find entry " + bucketName + " in directory " + S3_BASE_URI);
+ }
+
+ // Check the concrete directory file status
+ try {
+ final FileStatus directoryFileStatus = fs.getFileStatus(bucketPath);
+ assertTrue(directoryFileStatus.isDir());
+ assertEquals(0L, directoryFileStatus.getAccessTime());
+ assertTrue(directoryFileStatus.getModificationTime() > 0L);
+
+ } catch (FileNotFoundException e) {
+ fail(e.getMessage());
+ }
+
+ // Delete the bucket
+ fs.delete(bucketPath, true);
+
+ // Make sure the bucket no longer exists
+ try {
+ fs.getFileStatus(bucketPath);
+ fail("Expected FileNotFoundException for " + bucketPath.toUri());
+ } catch (FileNotFoundException e) {
+ // This is an expected exception
+ }
+
+ } catch (IOException ioe) {
+ fail(ioe.getMessage());
+ }
+ }
+
+ /**
+ * Creates and reads the a larger test file in S3. The test file is generated according to a specific pattern.
+ * During the read phase the incoming data stream is also checked against this pattern.
+ */
+ @Test
+ public void createAndReadLargeFileTest() {
+
+ try {
+ createAndReadFileTest(LARGE_FILE_SIZE);
+ } catch (IOException ioe) {
+ fail(ioe.getMessage());
+ }
+ }
+
+ /**
+ * Creates and reads the a small test file in S3. The test file is generated according to a specific pattern.
+ * During the read phase the incoming data stream is also checked against this pattern.
+ */
+ @Test
+ public void createAndReadSmallFileTest() {
+
+ try {
+ createAndReadFileTest(SMALL_FILE_SIZE);
+ } catch (IOException ioe) {
+ fail(ioe.getMessage());
+ }
+ }
+
+ /**
+ * The tests checks the mapping of the file system directory structure to the underlying bucket/object model of
+ * Amazon S3.
+ */
+ @Test
+ public void multiLevelDirectoryTest() {
+
+ if (!testActivated()) {
+ return;
+ }
+
+ final String dirName = getRandomName();
+ final String subdirName = getRandomName();
+ final String subsubdirName = getRandomName();
+ final String fileName = getRandomName();
+ final Path dir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR);
+ final Path subdir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + subdirName + Path.SEPARATOR);
+ final Path subsubdir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + subdirName + Path.SEPARATOR
+ + subsubdirName + Path.SEPARATOR);
+ final Path file = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + subdirName + Path.SEPARATOR + fileName);
+
+ try {
+
+ final FileSystem fs = dir.getFileSystem();
+
+ fs.mkdirs(subsubdir);
+
+ final OutputStream os = fs.create(file, true);
+ generateTestData(os, SMALL_FILE_SIZE);
+ os.close();
+
+ // On this directory levels there should only be one subdirectory
+ FileStatus[] list = fs.listStatus(dir);
+ int numberOfDirs = 0;
+ int numberOfFiles = 0;
+ for (final FileStatus entry : list) {
+
+ if (entry.isDir()) {
+ ++numberOfDirs;
+ assertEquals(subdir, entry.getPath());
+ } else {
+ fail(entry.getPath() + " is a file which must not appear on this directory level");
+ }
+ }
+
+ assertEquals(1, numberOfDirs);
+ assertEquals(0, numberOfFiles);
+
+ list = fs.listStatus(subdir);
+ numberOfDirs = 0;
+
+ for (final FileStatus entry : list) {
+ if (entry.isDir()) {
+ assertEquals(subsubdir, entry.getPath());
+ ++numberOfDirs;
+ } else {
+ assertEquals(file, entry.getPath());
+ ++numberOfFiles;
+ }
+ }
+
+ assertEquals(1, numberOfDirs);
+ assertEquals(1, numberOfFiles);
+
+ fs.delete(dir, true);
+
+ } catch (IOException ioe) {
+ fail(ioe.getMessage());
+ }
+ }
+
+ /**
+ * This test checks the S3 implementation of the file system method to retrieve the block locations of a file.
+ */
+ @Test
+ public void blockLocationTest() {
+
+ if (!testActivated()) {
+ return;
+ }
+
+ final String dirName = getRandomName();
+ final String fileName = getRandomName();
+ final Path dir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR);
+ final Path file = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + fileName);
+
+ try {
+
+ final FileSystem fs = dir.getFileSystem();
+
+ fs.mkdirs(dir);
+
+ final OutputStream os = fs.create(file, true);
+ generateTestData(os, SMALL_FILE_SIZE);
+ os.close();
+
+ final FileStatus fileStatus = fs.getFileStatus(file);
+ assertNotNull(fileStatus);
+
+ BlockLocation[] blockLocations = fs.getFileBlockLocations(fileStatus, 0, SMALL_FILE_SIZE + 1);
+ assertNull(blockLocations);
+
+ blockLocations = fs.getFileBlockLocations(fileStatus, 0, SMALL_FILE_SIZE);
+ assertEquals(1, blockLocations.length);
+
+ final BlockLocation bl = blockLocations[0];
+ assertNotNull(bl.getHosts());
+ assertEquals(1, bl.getHosts().length);
+ assertEquals(SMALL_FILE_SIZE, bl.getLength());
+ assertEquals(0, bl.getOffset());
+ final URI s3Uri = fs.getUri();
+ assertNotNull(s3Uri);
+ assertEquals(s3Uri.getHost(), bl.getHosts()[0]);
+
+ fs.delete(dir, true);
+
+ } catch (IOException ioe) {
+ fail(ioe.getMessage());
+ }
+ }
+
+ /**
+ * Creates and reads a file with the given size in S3. The test file is generated according to a specific pattern.
+ * During the read phase the incoming data stream is also checked against this pattern.
+ *
+ * @param fileSize
+ * the size of the file to be generated in bytes
+ * @throws IOException
+ * thrown if an I/O error occurs while writing or reading the test file
+ */
+ private void createAndReadFileTest(final int fileSize) throws IOException {
+
+ if (!testActivated()) {
+ return;
+ }
+
+ final String bucketName = getRandomName();
+ final String objectName = getRandomName();
+ final Path bucketPath = new Path(S3_BASE_URI + bucketName + Path.SEPARATOR);
+ final Path objectPath = new Path(S3_BASE_URI + bucketName + Path.SEPARATOR + objectName);
+
+ FileSystem fs = bucketPath.getFileSystem();
+
+ // Create test bucket
+ fs.mkdirs(bucketPath);
+
+ // Write test file to S3
+ final FSDataOutputStream outputStream = fs.create(objectPath, false);
+ generateTestData(outputStream, fileSize);
+ outputStream.close();
+
+ // Now read the same file back from S3
+ final FSDataInputStream inputStream = fs.open(objectPath);
+ testReceivedData(inputStream, fileSize);
+ inputStream.close();
+
+ // Delete test bucket
+ fs.delete(bucketPath, true);
+ }
+
+ /**
+ * Receives test data from the given input stream and checks the size of the data as well as the pattern inside the
+ * received data.
+ *
+ * @param inputStream
+ * the input stream to read the test data from
+ * @param expectedSize
+ * the expected size of the data to be read from the input stream in bytes
+ * @throws IOException
+ * thrown if an error occurs while reading the data
+ */
+ private void testReceivedData(final InputStream inputStream, final int expectedSize) throws IOException {
+
+ final byte[] testBuffer = new byte[TEST_BUFFER_SIZE];
+
+ int totalBytesRead = 0;
+ int nextExpectedNumber = 0;
+ while (true) {
+
+ final int bytesRead = inputStream.read(testBuffer);
+ if (bytesRead < 0) {
+ break;
+ }
+
+ totalBytesRead += bytesRead;
+
+ for (int i = 0; i < bytesRead; ++i) {
+ if (testBuffer[i] != nextExpectedNumber) {
+ throw new IOException("Read number " + testBuffer[i] + " but expected " + nextExpectedNumber);
+ }
+
+ ++nextExpectedNumber;
+
+ if (nextExpectedNumber == MODULUS) {
+ nextExpectedNumber = 0;
+ }
+ }
+ }
+
+ if (totalBytesRead != expectedSize) {
+ throw new IOException("Expected to read " + expectedSize + " bytes but only received " + totalBytesRead);
+ }
+ }
+
+ /**
+ * Generates test data of the given size according to some specific pattern and writes it to the provided output
+ * stream.
+ *
+ * @param outputStream
+ * the output stream to write the data to
+ * @param size
+ * the size of the test data to be generated in bytes
+ * @throws IOException
+ * thrown if an error occurs while writing the data
+ */
+ private void generateTestData(final OutputStream outputStream, final int size) throws IOException {
+
+ final byte[] testBuffer = new byte[TEST_BUFFER_SIZE];
+ for (int i = 0; i < testBuffer.length; ++i) {
+ testBuffer[i] = (byte) (i % MODULUS);
+ }
+
+ int bytesWritten = 0;
+ while (bytesWritten < size) {
+
+ final int diff = size - bytesWritten;
+ if (diff < testBuffer.length) {
+ outputStream.write(testBuffer, 0, diff);
+ bytesWritten += diff;
+ } else {
+ outputStream.write(testBuffer);
+ bytesWritten += testBuffer.length;
+ }
+ }
+ }
+
+ /**
+ * Generates a random name.
+ *
+ * @return a random name
+ */
+ private String getRandomName() {
+
+ final StringBuilder stringBuilder = new StringBuilder();
+ for (int i = 0; i < NAME_LENGTH; ++i) {
+ final char c = ALPHABET[(int) (Math.random() * (double) ALPHABET.length)];
+ stringBuilder.append(c);
+ }
+
+ return stringBuilder.toString();
+ }
+
+ /**
+ * Checks whether the AWS access key and the AWS secret keys have been successfully loaded from the configuration
+ * and whether the S3 tests shall be performed.
+ *
+ * @return <code>true</code> if the tests shall be performed, <code>false</code> if the tests shall be skipped
+ * because at least one AWS key is missing
+ */
+ private boolean testActivated() {
+
+ final String accessKey = GlobalConfiguration.getString(S3FileSystem.S3_ACCESS_KEY_KEY, null);
+ final String secretKey = GlobalConfiguration.getString(S3FileSystem.S3_SECRET_KEY_KEY, null);
+
+ if (accessKey != null && secretKey != null) {
+ return true;
+ }
+
+ return false;
+ }
+}
[06/30] Replace custom Java NIO TCP/IP code with Netty 4 library
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ChannelManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ChannelManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ChannelManager.java
index 405d79e..7b01f4e 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ChannelManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ChannelManager.java
@@ -31,15 +31,13 @@ import eu.stratosphere.nephele.AbstractID;
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.network.bufferprovider.DiscardBufferPool;
import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool;
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.InputGate;
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.LogFactory;
@@ -52,7 +50,7 @@ import java.util.concurrent.ConcurrentHashMap;
/**
* 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);
@@ -68,20 +66,27 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
private final GlobalBufferPool globalBufferPool;
- private final NetworkConnectionManager networkConnectionManager;
+ private final NettyConnectionManager nettyConnectionManager;
private final InetSocketAddress ourAddress;
- private final SerialSingleBufferPool discardingDataPool;
+ private final DiscardBufferPool discardBufferPool;
// -----------------------------------------------------------------------------------------------------------------
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.connectionInfo = connectionInfo;
+
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
this.channels = new ConcurrentHashMap<ChannelID, Channel>();
@@ -91,11 +96,11 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
this.ourAddress = new InetSocketAddress(connectionInfo.address(), connectionInfo.dataPort());
// a special pool if the data is to be discarded
- this.discardingDataPool = new SerialSingleBufferPool(networkBufferSize);
+ this.discardBufferPool = new DiscardBufferPool();
}
public void shutdown() {
- this.networkConnectionManager.shutDown();
+ this.nettyConnectionManager.shutdown();
this.globalBufferPool.destroy();
}
@@ -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());
if (channel == null) {
LOG.error("Cannot find channel for channel ID " + envelope.getSource());
@@ -319,7 +324,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
final RemoteReceiver ourAddress = new RemoteReceiver(this.ourAddress, connectionIndex);
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
* 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
* @throws IOException
- * @throws InterruptedException
*/
private EnvelopeReceiverList getReceiverList(JobID jobID, ChannelID sourceChannelID, boolean reportException) throws IOException {
EnvelopeReceiverList receiverList = this.receiverCache.get(sourceChannelID);
@@ -383,8 +387,10 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
this.receiverCache.put(sourceChannelID, receiverList);
if (LOG.isDebugEnabled()) {
- LOG.debug("Receivers for source channel ID " + sourceChannelID + " at task manager " + this.connectionInfo +
- ": " + receiverList);
+ LOG.debug(String.format("Receiver for %s: %s [%s])",
+ sourceChannelID,
+ receiverList.hasLocalReceiver() ? receiverList.getLocalReceiver() : receiverList.getRemoteReceiver(),
+ receiverList.hasLocalReceiver() ? "local" : "remote"));
}
return receiverList;
@@ -436,7 +442,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
} catch (InterruptedException e) {
throw new IOException(e.getMessage());
}
-
+
srcBuffer.copyToBuffer(destBuffer);
envelope.setBuffer(destBuffer);
srcBuffer.recycleBuffer();
@@ -453,7 +459,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
generateSenderHint(envelope, remoteReceiver);
}
- this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, envelope);
+ this.nettyConnectionManager.enqueue(envelope, remoteReceiver);
success = true;
}
} finally {
@@ -501,7 +507,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
generateSenderHint(envelope, remoteReceiver);
}
- this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, envelope);
+ this.nettyConnectionManager.enqueue(envelope, remoteReceiver);
}
}
@@ -597,7 +603,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
// check if the receiver is already gone
if (receiverList == null) {
- return this.discardingDataPool;
+ return this.discardBufferPool;
}
if (!receiverList.hasLocalReceiver() || receiverList.hasRemoteReceiver()) {
@@ -610,7 +616,7 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
if (channel == null) {
// receiver is already canceled
- return this.discardingDataPool;
+ return this.discardBufferPool;
}
if (!channel.isInputChannel()) {
@@ -633,8 +639,6 @@ public final class ChannelManager implements EnvelopeDispatcher, BufferProviderB
bufferPool.logBufferUtilization();
}
- this.networkConnectionManager.logBufferUtilization();
-
System.out.println("\tIncoming connections:");
for (Channel channel : this.channels.values()) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/Envelope.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/Envelope.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/Envelope.java
new file mode 100644
index 0000000..0f8bae8
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/Envelope.java
@@ -0,0 +1,178 @@
+/***********************************************************************************************************************
+ * 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;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.runtime.io.serialization.DataInputDeserializer;
+import eu.stratosphere.runtime.io.serialization.DataOutputSerializer;
+import eu.stratosphere.util.InstantiationUtil;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public final class Envelope {
+
+ private final JobID jobID;
+
+ private final ChannelID source;
+
+ private final int sequenceNumber;
+
+ private ByteBuffer serializedEventList;
+
+ private Buffer buffer;
+
+ public Envelope(int sequenceNumber, JobID jobID, ChannelID source) {
+ this.sequenceNumber = sequenceNumber;
+ this.jobID = jobID;
+ this.source = source;
+ }
+
+ private Envelope(Envelope toDuplicate) {
+ this.jobID = toDuplicate.jobID;
+ this.source = toDuplicate.source;
+ this.sequenceNumber = toDuplicate.sequenceNumber;
+ this.serializedEventList = null;
+ this.buffer = null;
+ }
+
+ public Envelope duplicate() {
+ Envelope duplicate = new Envelope(this);
+ if (hasBuffer()) {
+ duplicate.setBuffer(this.buffer.duplicate());
+ }
+
+ return duplicate;
+ }
+
+ public Envelope duplicateWithoutBuffer() {
+ return new Envelope(this);
+ }
+
+ public JobID getJobID() {
+ return this.jobID;
+ }
+
+ public ChannelID getSource() {
+ return this.source;
+ }
+
+ public int getSequenceNumber() {
+ return this.sequenceNumber;
+ }
+
+ public void setEventsSerialized(ByteBuffer serializedEventList) {
+ if (this.serializedEventList != null) {
+ throw new IllegalStateException("Event list has already been set.");
+ }
+
+ this.serializedEventList = serializedEventList;
+ }
+
+ public void serializeEventList(List<? extends AbstractEvent> eventList) {
+ if (this.serializedEventList != null) {
+ throw new IllegalStateException("Event list has already been set.");
+ }
+
+ this.serializedEventList = serializeEvents(eventList);
+ }
+
+ public ByteBuffer getEventsSerialized() {
+ return this.serializedEventList;
+ }
+
+ public List<? extends AbstractEvent> deserializeEvents() {
+ return deserializeEvents(getClass().getClassLoader());
+ }
+
+ public List<? extends AbstractEvent> deserializeEvents(ClassLoader classloader) {
+ if (this.serializedEventList == null) {
+ return Collections.emptyList();
+ }
+
+ try {
+ DataInputDeserializer deserializer = new DataInputDeserializer(this.serializedEventList);
+
+ int numEvents = deserializer.readInt();
+ ArrayList<AbstractEvent> events = new ArrayList<AbstractEvent>(numEvents);
+
+ for (int i = 0; i < numEvents; i++) {
+ String className = deserializer.readUTF();
+ Class<? extends AbstractEvent> clazz;
+ try {
+ clazz = Class.forName(className).asSubclass(AbstractEvent.class);
+ } catch (ClassNotFoundException e) {
+ throw new RuntimeException("Could not load event class '" + className + "'.", e);
+ } catch (ClassCastException e) {
+ throw new RuntimeException("The class '" + className + "' is no valid subclass of '" + AbstractEvent.class.getName() + "'.", e);
+ }
+
+ AbstractEvent evt = InstantiationUtil.instantiate(clazz, AbstractEvent.class);
+ evt.read(deserializer);
+
+ events.add(evt);
+ }
+
+ return events;
+ }
+ catch (IOException e) {
+ throw new RuntimeException("Error while deserializing the events.", e);
+ }
+ }
+
+ public void setBuffer(Buffer buffer) {
+ this.buffer = buffer;
+ }
+
+ public Buffer getBuffer() {
+ return this.buffer;
+ }
+
+ private ByteBuffer serializeEvents(List<? extends AbstractEvent> events) {
+ try {
+ // create the serialized event list
+ DataOutputSerializer serializer = events.size() == 0
+ ? new DataOutputSerializer(4)
+ : new DataOutputSerializer(events.size() * 32);
+ serializer.writeInt(events.size());
+
+ for (AbstractEvent evt : events) {
+ serializer.writeUTF(evt.getClass().getName());
+ evt.write(serializer);
+ }
+
+ return serializer.wrapAsByteBuffer();
+ }
+ catch (IOException e) {
+ throw new RuntimeException("Error while serializing the task events.", e);
+ }
+ }
+
+ public boolean hasBuffer() {
+ 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());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeDispatcher.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeDispatcher.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeDispatcher.java
new file mode 100644
index 0000000..372ff92
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeDispatcher.java
@@ -0,0 +1,46 @@
+/***********************************************************************************************************************
+ * 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;
+
+import java.io.IOException;
+
+/**
+ * A envelope dispatcher receives {@link Envelope}s and sends them to all of its destinations.
+ */
+public interface EnvelopeDispatcher {
+
+ /**
+ * Dispatches an envelope from an output channel to the receiving input channels (forward flow).
+ *
+ * @param envelope envelope to be sent
+ */
+ void dispatchFromOutputChannel(Envelope envelope) throws IOException, InterruptedException;
+
+ /**
+ * Dispatches an envelope from an input channel to the receiving output channels (backwards flow).
+ *
+ * @param envelope envelope to be sent
+ */
+ void dispatchFromInputChannel(Envelope envelope) throws IOException, InterruptedException;
+
+ /**
+ * Dispatches an envelope from an incoming TCP connection.
+ * <p>
+ * After an envelope has been constructed from a TCP socket, this method is called to send the envelope to the
+ * receiving input channel.
+ *
+ * @param envelope envelope to be sent
+ */
+ void dispatchFromNetwork(Envelope envelope) throws IOException, InterruptedException;
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeReceiverList.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeReceiverList.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeReceiverList.java
new file mode 100644
index 0000000..d53d728
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/EnvelopeReceiverList.java
@@ -0,0 +1,75 @@
+/***********************************************************************************************************************
+ * 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 java.net.InetAddress;
+
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.network.ConnectionInfoLookupResponse;
+import eu.stratosphere.runtime.io.network.RemoteReceiver;
+
+/**
+ * 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
+ * {@link InetAddress} objects and finally checkpoints which are identified by
+ * <p>
+ * This class is thread-safe.
+ *
+ */
+public class EnvelopeReceiverList {
+
+ private final ChannelID localReceiver;
+
+ private final RemoteReceiver remoteReceiver;
+
+ public EnvelopeReceiverList(ConnectionInfoLookupResponse cilr) {
+ this.localReceiver = cilr.getLocalTarget();
+ this.remoteReceiver = cilr.getRemoteTarget();
+ }
+
+ public EnvelopeReceiverList(ChannelID localReceiver) {
+ this.localReceiver = localReceiver;
+ this.remoteReceiver = null;
+ }
+
+ public EnvelopeReceiverList(RemoteReceiver remoteReceiver) {
+ this.localReceiver = null;
+ this.remoteReceiver = remoteReceiver;
+ }
+
+ public boolean hasLocalReceiver() {
+ return this.localReceiver != null;
+ }
+
+ public boolean hasRemoteReceiver() {
+ return this.remoteReceiver != null;
+ }
+
+ public int getTotalNumberOfReceivers() {
+ return (this.localReceiver == null ? 0 : 1) + (this.remoteReceiver == null ? 0 : 1);
+ }
+
+ public RemoteReceiver getRemoteReceiver() {
+ return this.remoteReceiver;
+ }
+
+ public ChannelID getLocalReceiver() {
+ return this.localReceiver;
+ }
+
+ @Override
+ public String toString() {
+ return "local receiver: " + this.localReceiver + ", remote receiver: " + this.remoteReceiver;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java
deleted file mode 100644
index 44ec642..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/***********************************************************************************************************************
- * 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());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java
index 32be058..65287f7 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java
@@ -21,7 +21,6 @@ import java.util.List;
import eu.stratosphere.nephele.event.task.AbstractEvent;
import eu.stratosphere.runtime.io.channels.ChannelID;
-import eu.stratosphere.runtime.io.network.envelope.Envelope;
public final class SenderHintEvent extends AbstractEvent {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java
index 1d23e93..af08aa8 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java
@@ -1,5 +1,5 @@
/***********************************************************************************************************************
- * 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
* the License. You may obtain a copy of the License at
@@ -13,16 +13,19 @@
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
- * {@link eu.stratosphere.runtime.io.Buffer} has
- * become available again.
- *
+ * This interface must be implemented to receive an asynchronous callback from
+ * a {@link BufferProvider} as soon as a buffer has become available again.
*/
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;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
index e3085ee..d82b427 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
@@ -65,5 +65,21 @@ public interface BufferProvider {
* @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
*/
- 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;
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java
new file mode 100644
index 0000000..5daa509
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java
@@ -0,0 +1,51 @@
+/***********************************************************************************************************************
+ * 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.bufferprovider;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.BufferRecycler;
+
+public final class DiscardBufferPool implements BufferProvider, BufferRecycler {
+
+ @Override
+ public Buffer requestBuffer(int minBufferSize) {
+ return null;
+ }
+
+ @Override
+ public Buffer requestBufferBlocking(int minBufferSize) {
+ return null;
+ }
+
+ @Override
+ public int getBufferSize() {
+ return 0;
+ }
+
+ @Override
+ public void reportAsynchronousEvent() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+ return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
+ }
+
+ @Override
+ public void recycle(MemorySegment buffer) {
+ throw new UnsupportedOperationException();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
index e8aeb11..3eb10c1 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
@@ -184,20 +184,20 @@ public final class LocalBufferPool implements BufferProvider {
}
@Override
- public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+ public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
synchronized (this.buffers) {
if (!this.buffers.isEmpty()) {
- return false;
+ return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_AVAILABLE;
}
if (this.isDestroyed) {
- return false;
+ return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
}
this.listeners.add(listener);
}
- return true;
+ return BufferAvailabilityRegistration.REGISTERED;
}
/**
@@ -221,7 +221,7 @@ public final class LocalBufferPool implements BufferProvider {
}
this.globalBufferPool.returnBuffer(this.buffers.poll());
- this.numRequestedBuffers --;
+ this.numRequestedBuffers--;
}
this.buffers.notify();
@@ -294,11 +294,17 @@ public final class LocalBufferPool implements BufferProvider {
this.globalBufferPool.returnBuffer(buffer);
this.numRequestedBuffers--;
} else {
- this.buffers.add(buffer);
- this.buffers.notify();
-
- while (!this.listeners.isEmpty()) {
- this.listeners.poll().bufferAvailable();
+ 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 {
+ this.buffers.add(buffer);
+ this.buffers.notify();
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java
deleted file mode 100644
index 217a5ce..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/***********************************************************************************************************************
- * 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.bufferprovider;
-
-import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.runtime.io.Buffer;
-import eu.stratosphere.runtime.io.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
- public Buffer requestBuffer(int minBufferSize) {
- if (minBufferSize <= this.bufferSize) {
- return this.buffer.duplicate();
- }
- else {
- throw new IllegalArgumentException("Requesting buffer with size " + minBufferSize + ". Pool's buffer size is " + this.bufferSize);
- }
- }
-
- @Override
- public Buffer requestBufferBlocking(int minBufferSize) {
- if (minBufferSize <= this.bufferSize) {
- return this.buffer.duplicate();
- }
- else {
- throw new IllegalArgumentException("Requesting buffer with size " + minBufferSize + ". Pool's buffer size is " + this.bufferSize);
- }
- }
-
- @Override
- public int getBufferSize() {
- return this.bufferSize;
- }
-
- @Override
- public void reportAsynchronousEvent() {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public void recycle(MemorySegment buffer) {}
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java
deleted file mode 100644
index a692aec..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/***********************************************************************************************************************
- * 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.event.task.AbstractEvent;
-import eu.stratosphere.runtime.io.Buffer;
-import eu.stratosphere.runtime.io.channels.ChannelID;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.runtime.io.serialization.DataInputDeserializer;
-import eu.stratosphere.runtime.io.serialization.DataOutputSerializer;
-import eu.stratosphere.util.InstantiationUtil;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public final class Envelope {
-
- private final JobID jobID;
-
- private final ChannelID source;
-
- private final int sequenceNumber;
-
- private ByteBuffer serializedEventList;
-
- private Buffer buffer;
-
- public Envelope(int sequenceNumber, JobID jobID, ChannelID source) {
- this.sequenceNumber = sequenceNumber;
- this.jobID = jobID;
- this.source = source;
- }
-
- private Envelope(Envelope toDuplicate) {
- this.jobID = toDuplicate.jobID;
- this.source = toDuplicate.source;
- this.sequenceNumber = toDuplicate.sequenceNumber;
- this.serializedEventList = null;
- this.buffer = null;
- }
-
- public Envelope duplicate() {
- Envelope duplicate = new Envelope(this);
- if (hasBuffer()) {
- duplicate.setBuffer(this.buffer.duplicate());
- }
-
- return duplicate;
- }
-
- public Envelope duplicateWithoutBuffer() {
- return new Envelope(this);
- }
-
- public JobID getJobID() {
- return this.jobID;
- }
-
- public ChannelID getSource() {
- return this.source;
- }
-
- public int getSequenceNumber() {
- return this.sequenceNumber;
- }
-
- public void setEventsSerialized(ByteBuffer serializedEventList) {
- if (this.serializedEventList != null)
- throw new IllegalStateException("Event list has already been set.");
-
- this.serializedEventList = serializedEventList;
- }
-
- public void serializeEventList(List<? extends AbstractEvent> eventList) {
- if (this.serializedEventList != null)
- throw new IllegalStateException("Event list has already been set.");
-
- this.serializedEventList = serializeEvents(eventList);
- }
-
- public ByteBuffer getEventsSerialized() {
- return this.serializedEventList;
- }
-
- public List<? extends AbstractEvent> deserializeEvents() {
- return deserializeEvents(getClass().getClassLoader());
- }
-
- public List<? extends AbstractEvent> deserializeEvents(ClassLoader classloader) {
- if (this.serializedEventList == null) {
- return Collections.emptyList();
- }
-
- try {
- DataInputDeserializer deserializer = new DataInputDeserializer(this.serializedEventList);
-
- int numEvents = deserializer.readInt();
- ArrayList<AbstractEvent> events = new ArrayList<AbstractEvent>(numEvents);
-
- for (int i = 0; i < numEvents; i++) {
- String className = deserializer.readUTF();
- Class<? extends AbstractEvent> clazz;
- try {
- clazz = Class.forName(className).asSubclass(AbstractEvent.class);
- } catch (ClassNotFoundException e) {
- throw new RuntimeException("Could not load event class '" + className + "'.", e);
- } catch (ClassCastException e) {
- throw new RuntimeException("The class '" + className + "' is no valid subclass of '" + AbstractEvent.class.getName() + "'.", e);
- }
-
- AbstractEvent evt = InstantiationUtil.instantiate(clazz, AbstractEvent.class);
- evt.read(deserializer);
-
- events.add(evt);
- }
-
- return events;
- }
- catch (IOException e) {
- throw new RuntimeException("Error while deserializing the events.", e);
- }
- }
-
- public void setBuffer(Buffer buffer) {
- this.buffer = buffer;
- }
-
- public Buffer getBuffer() {
- return this.buffer;
- }
-
- private ByteBuffer serializeEvents(List<? extends AbstractEvent> events) {
- try {
- // create the serialized event list
- DataOutputSerializer serializer = events.size() == 0
- ? new DataOutputSerializer(4)
- : new DataOutputSerializer(events.size() * 32);
- serializer.writeInt(events.size());
-
- for (AbstractEvent evt : events) {
- serializer.writeUTF(evt.getClass().getName());
- evt.write(serializer);
- }
-
- return serializer.wrapAsByteBuffer();
- }
- catch (IOException e) {
- throw new RuntimeException("Error while serializing the task events.", e);
- }
- }
-
- public boolean hasBuffer() {
- return this.buffer != null;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java
deleted file mode 100644
index 2b69c0d..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/***********************************************************************************************************************
- * 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 java.io.IOException;
-
-/**
- * A envelope dispatcher receives {@link Envelope}s and sends them to all of its destinations.
- */
-public interface EnvelopeDispatcher {
-
- /**
- * Dispatches an envelope from an output channel to the receiving input channels (forward flow).
- *
- * @param envelope envelope to be sent
- */
- void dispatchFromOutputChannel(Envelope envelope) throws IOException, InterruptedException;
-
- /**
- * Dispatches an envelope from an input channel to the receiving output channels (backwards flow).
- *
- * @param envelope envelope to be sent
- */
- void dispatchFromInputChannel(Envelope envelope) throws IOException, InterruptedException;
-
- /**
- * Dispatches an envelope from an incoming TCP connection.
- * <p>
- * After an envelope has been constructed from a TCP socket, this method is called to send the envelope to the
- * receiving input channel.
- *
- * @param envelope envelope to be sent
- */
- void dispatchFromNetwork(Envelope envelope) throws IOException, InterruptedException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java
deleted file mode 100644
index 7b7e178..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/***********************************************************************************************************************
- * 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;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.java
deleted file mode 100644
index f99e1f2..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/***********************************************************************************************************************
- * 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 java.net.InetAddress;
-
-import eu.stratosphere.runtime.io.channels.ChannelID;
-import eu.stratosphere.runtime.io.network.ConnectionInfoLookupResponse;
-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
- * receivers: Local receivers identified by {@link ChannelID} objects, remote receivers identified by
- * {@link InetAddress} objects and finally checkpoints which are identified by
- * <p>
- * This class is thread-safe.
- *
- */
-public class EnvelopeReceiverList {
-
- private final ChannelID localReceiver;
-
- private final RemoteReceiver remoteReceiver;
-
- public EnvelopeReceiverList(ConnectionInfoLookupResponse cilr) {
- this.localReceiver = cilr.getLocalTarget();
- this.remoteReceiver = cilr.getRemoteTarget();
- }
-
- public EnvelopeReceiverList(ChannelID localReceiver) {
- this.localReceiver = localReceiver;
- this.remoteReceiver = null;
- }
-
- public EnvelopeReceiverList(RemoteReceiver remoteReceiver) {
- this.localReceiver = null;
- this.remoteReceiver = remoteReceiver;
- }
-
- public boolean hasLocalReceiver() {
- return this.localReceiver != null;
- }
-
- public boolean hasRemoteReceiver() {
- return this.remoteReceiver != null;
- }
-
- public int getTotalNumberOfReceivers() {
- return (this.localReceiver == null ? 0 : 1) + (this.remoteReceiver == null ? 0 : 1);
- }
-
- public RemoteReceiver getRemoteReceiver() {
- return this.remoteReceiver;
- }
-
- public ChannelID getLocalReceiver() {
- return this.localReceiver;
- }
-
- @Override
- public String toString() {
- return "local receiver: " + this.localReceiver + ", remote receiver: " + this.remoteReceiver;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java
deleted file mode 100644
index c00e61b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/***********************************************************************************************************************
- * 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();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.java
deleted file mode 100644
index f7d49bf..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/***********************************************************************************************************************
- * 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.runtime.io.network.bufferprovider.BufferProvider;
-
-/**
- * This exception is thrown to indicate that the deserialization process of a {@link Envelope} could not be
- * continued because a {@link Buffer} to store the envelope's content is currently not available.
- *
- */
-public final class NoBufferAvailableException extends Exception {
-
- /**
- * Generated serial UID.
- */
- 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
- * the buffer provider which could not deliver a buffer
- */
- public NoBufferAvailableException(final BufferProvider bufferProvider) {
- this.bufferProvider = bufferProvider;
- }
-
- /**
- * Returns the buffer provider which could not deliver a buffer.
- *
- * @return the buffer provider which could not deliver a buffer
- */
- public BufferProvider getBufferProvider() {
- return this.bufferProvider;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
new file mode 100644
index 0000000..1ab1871
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
@@ -0,0 +1,344 @@
+/***********************************************************************************************************************
+ * 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;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDispatcherHandler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDispatcherHandler.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDispatcherHandler.java
new file mode 100644
index 0000000..d0270b6
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDispatcherHandler.java
@@ -0,0 +1,41 @@
+/***********************************************************************************************************************
+ * 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);
+ }
+}
[24/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionJob.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionJob.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionJob.java
deleted file mode 100644
index d9c09d3..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionJob.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/***********************************************************************************************************************
- * 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.example.union;
-
-import java.io.File;
-import java.io.IOException;
-
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.client.JobClient;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.jobgraph.JobFileInputVertex;
-import eu.stratosphere.nephele.jobgraph.JobFileOutputVertex;
-import eu.stratosphere.nephele.jobgraph.JobGraph;
-import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
-import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
-import eu.stratosphere.nephele.util.JarFileCreator;
-
-public class UnionJob {
-
- public static void main(final String[] args) {
-
- // Create graph and define vertices
- final JobGraph unionGraph = new JobGraph("Union Job");
-
- final JobFileInputVertex input1 = new JobFileInputVertex("Input 1", unionGraph);
- input1.setFileInputClass(ProducerTask.class);
- input1.setFilePath(new Path("file:///tmp/"));
-
- final JobFileInputVertex input2 = new JobFileInputVertex("Input 2", unionGraph);
- input2.setFileInputClass(ProducerTask.class);
- input2.setFilePath(new Path("file:///tmp/"));
-
- final JobTaskVertex union = new JobTaskVertex("Union", unionGraph);
- union.setTaskClass(UnionTask.class);
-
- final JobFileOutputVertex output = new JobFileOutputVertex("Output", unionGraph);
- output.setFileOutputClass(ConsumerTask.class);
- output.setFilePath(new Path("file:///tmp/"));
-
- // Create edges between vertices
- try {
- input1.connectTo(union, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
- input2.connectTo(union, ChannelType.NETWORK, DistributionPattern.POINTWISE);
- union.connectTo(output, ChannelType.INMEMORY);
- } catch (JobGraphDefinitionException e) {
- e.printStackTrace();
- return;
- }
-
- // Create jar file and attach it
- final File jarFile = new File("/tmp/unionJob.jar");
- final JarFileCreator jarFileCreator = new JarFileCreator(jarFile);
- jarFileCreator.addClass(ProducerTask.class);
- jarFileCreator.addClass(UnionTask.class);
- jarFileCreator.addClass(ConsumerTask.class);
-
- try {
- jarFileCreator.createJarFile();
- } catch (IOException ioe) {
-
- ioe.printStackTrace();
-
- if (jarFile.exists()) {
- jarFile.delete();
- }
-
- return;
- }
-
- //Define instance sharing
- input1.setVertexToShareInstancesWith(output);
- input2.setVertexToShareInstancesWith(output);
- union.setVertexToShareInstancesWith(output);
-
- unionGraph.addJar(new Path("file://" + jarFile.getAbsolutePath()));
-
- final Configuration conf = new Configuration();
- conf.setString("jobmanager.rpc.address", "localhost");
-
- try {
- final JobClient jobClient = new JobClient(unionGraph, conf);
- jobClient.submitJobAndWait();
- } catch (Exception e) {
- e.printStackTrace();
- }
-
- if (jarFile.exists()) {
- jarFile.delete();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionTask.java
deleted file mode 100644
index d99eb42..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/UnionTask.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/***********************************************************************************************************************
- * 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.example.union;
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.MutableRecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.io.UnionRecordReader;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-public class UnionTask extends AbstractTask {
-
- private UnionRecordReader<StringRecord> input;
-
- private RecordWriter<StringRecord> output;
-
- @Override
- public void registerInputOutput() {
- @SuppressWarnings("unchecked")
- MutableRecordReader<StringRecord>[] recordReaders = (MutableRecordReader<StringRecord>[]) new MutableRecordReader<?>[2];
- recordReaders[0] = new MutableRecordReader<StringRecord>(this);
- recordReaders[1] = new MutableRecordReader<StringRecord>(this);
-
- this.input = new UnionRecordReader<StringRecord>(recordReaders, StringRecord.class);
- this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
- }
-
- @Override
- public void invoke() throws Exception {
- while (this.input.hasNext()) {
- this.output.emit(this.input.next());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java
new file mode 100644
index 0000000..6a09e89
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java
@@ -0,0 +1,25 @@
+/***********************************************************************************************************************
+ *
+ * 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.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.
+ */
+public class CancelTaskException extends RuntimeException {
+ private static final long serialVersionUID = 1L;
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
index a606dcf..e0bcc70 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
@@ -1,5 +1,5 @@
/***********************************************************************************************************************
- * 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
* the License. You may obtain a copy of the License at
@@ -13,30 +13,25 @@
package eu.stratosphere.nephele.execution;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.FutureTask;
-
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.core.fs.Path;
import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.ChannelSelector;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.RecordDeserializerFactory;
-import eu.stratosphere.nephele.io.channels.ChannelID;
+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.JobID;
import eu.stratosphere.nephele.protocols.AccumulatorProtocol;
import eu.stratosphere.nephele.services.iomanager.IOManager;
import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
import eu.stratosphere.nephele.template.InputSplitProvider;
/**
* The user code of every Nephele task runs inside an <code>Environment</code> object. The environment provides
* important services to the task. It keeps track of setting up the communication channels and provides access to input
* splits, memory manager, etc.
- *
*/
public interface Environment {
/**
@@ -49,9 +44,9 @@ public interface Environment {
JobID getJobID();
/**
- * Returns the task configuration object which was attached to the original {@link JobVertex}.
+ * Returns the task configuration object which was attached to the original JobVertex.
*
- * @return the task configuration object which was attached to the original {@link JobVertex}
+ * @return the task configuration object which was attached to the original JobVertex.
*/
Configuration getTaskConfiguration();
@@ -128,13 +123,6 @@ public interface Environment {
GateID getNextUnboundInputGateID();
/**
- * Returns the next unbound output gate ID or <code>null</code> if no such ID exists
- *
- * @return the next unbound output gate ID or <code>null</code> if no such ID exists
- */
- GateID getNextUnboundOutputGateID();
-
- /**
* Returns the number of output gates registered with this environment.
*
* @return the number of output gates registered with this environment
@@ -163,46 +151,16 @@ public interface Environment {
int getNumberOfInputChannels();
/**
- * Creates an output gate.
- *
- * @param gateID
- * @param outputClass
- * @param selector
- * @param isBroadcast
- * @param <T>
- * The type of the record consumed by the output gate.
- * @return The created output gate.
+ * Creates a new OutputGate and registers it with the Environment.
+ *
+ * @return the newly created output gate
*/
- <T extends IOReadableWritable> OutputGate<T> createOutputGate(GateID gateID, Class<T> outputClass,
- ChannelSelector<T> selector, boolean isBroadcast);
+ OutputGate createAndRegisterOutputGate();
/**
- * Creates an input gate.
- *
- * @param gateID
- * @param deserializer
- * @param distributionPattern
- * @param <T>
- * The type of the record read from the input gate.
- * @return The created input gate.
+ * Creates a new InputGate and registers it with the Environment.
*/
- <T extends IOReadableWritable> InputGate<T> createInputGate(GateID gateID, RecordDeserializerFactory<T> deserializerFactory);
-
- /**
- * Registers an output gate with this environment.
- *
- * @param outputGate
- * the output gate to be registered
- */
- void registerOutputGate(OutputGate<? extends IOReadableWritable> outputGate);
-
- /**
- * Registers an input gate with this environment.
- *
- * @param inputGate
- * the input gate to be registered
- */
- void registerInputGate(InputGate<? extends IOReadableWritable> inputGate);
+ <T extends IOReadableWritable> InputGate<T> createAndRegisterInputGate();
/**
* Returns the IDs of all output channels connected to this environment.
@@ -255,5 +213,13 @@ public interface Environment {
*/
AccumulatorProtocol getAccumulatorProtocolProxy();
- Map<String, FutureTask<Path>> getCopyTask();
+ /**
+ * Returns the buffer provider for this environment.
+ * <p>
+ * 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
+ * @see eu.stratosphere.runtime.io.api.RecordWriter
+ */
+ BufferProvider getOutputBufferProvider();
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
index 7d7cebb..55f036a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
@@ -97,29 +97,9 @@ public final class ExecutionStateTransition {
unexpectedStateChange = false;
}
- // This transition can appear if a task cannot be deployed at the assigned task manager.
- else if (oldState == ExecutionState.STARTING && newState == ExecutionState.FAILED) {
- unexpectedStateChange = false;
- }
-
// -------------- error cases --------------
- else if (newState == FAILED) {
- // any state may fail
- unexpectedStateChange = false;
- }
-
- // This is a regular transition as a result of a cancel operation.
- else if (oldState == ExecutionState.RUNNING && newState == ExecutionState.CANCELING) {
- unexpectedStateChange = false;
- }
-
- // This is a regular transition as a result of a cancel operation.
- else if (oldState == ExecutionState.FINISHING && newState == ExecutionState.CANCELING) {
- unexpectedStateChange = false;
- }
-
- // This is a regular transition as a result of a cancel operation.
- else if (oldState == ExecutionState.CANCELING && newState == ExecutionState.CANCELED) {
+ else if (newState == FAILED || newState == CANCELED || newState == CANCELING) {
+ // any state may fail or cancel itself
unexpectedStateChange = false;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
index a8ca11c..59787d2 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
@@ -36,21 +36,25 @@ import eu.stratosphere.nephele.deployment.ChannelDeploymentDescriptor;
import eu.stratosphere.nephele.deployment.GateDeploymentDescriptor;
import eu.stratosphere.nephele.deployment.TaskDeploymentDescriptor;
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.io.ChannelSelector;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.RecordDeserializerFactory;
-import eu.stratosphere.nephele.io.RuntimeInputGate;
-import eu.stratosphere.nephele.io.RuntimeOutputGate;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.nephele.protocols.AccumulatorProtocol;
import eu.stratosphere.nephele.services.iomanager.IOManager;
import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
import eu.stratosphere.nephele.template.AbstractInvokable;
import eu.stratosphere.nephele.template.InputSplitProvider;
+import eu.stratosphere.runtime.io.Buffer;
+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.gates.GateID;
+import eu.stratosphere.runtime.io.gates.InputGate;
+import eu.stratosphere.runtime.io.gates.OutputGate;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
+import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool;
+import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPool;
+import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPoolOwner;
import eu.stratosphere.util.StringUtils;
/**
@@ -60,7 +64,7 @@ import eu.stratosphere.util.StringUtils;
* <p>
* This class is thread-safe.
*/
-public class RuntimeEnvironment implements Environment, Runnable {
+public class RuntimeEnvironment implements Environment, BufferProvider, LocalBufferPoolOwner, Runnable {
/**
* The log object used for debugging.
@@ -75,7 +79,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
/**
* List of output gates created by the task.
*/
- private final List<OutputGate<? extends IOReadableWritable>> outputGates = new CopyOnWriteArrayList<OutputGate<? extends IOReadableWritable>>();
+ private final List<OutputGate> outputGates = new CopyOnWriteArrayList<OutputGate>();
/**
* List of input gates created by the task.
@@ -83,12 +87,6 @@ public class RuntimeEnvironment implements Environment, Runnable {
private final List<InputGate<? extends IOReadableWritable>> inputGates = new CopyOnWriteArrayList<InputGate<? extends IOReadableWritable>>();
/**
- * Queue of unbound output gate IDs which are required for deserializing an environment in the course of an RPC
- * call.
- */
- private final Queue<GateID> unboundOutputGateIDs = new ArrayDeque<GateID>();
-
- /**
* Queue of unbound input gate IDs which are required for deserializing an environment in the course of an RPC
* call.
*/
@@ -145,7 +143,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
private volatile ExecutionObserver executionObserver = null;
/**
- * The RPC procy to report accumulators to JobManager
+ * The RPC proxy to report accumulators to JobManager
*/
private AccumulatorProtocol accumulatorProtocolProxy = null;
@@ -164,7 +162,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
*/
private final String taskName;
- private Map<String,FutureTask<Path>> cacheCopyTasks = new HashMap<String, FutureTask<Path>>();
+ private LocalBufferPool outputBufferPool;
/**
* Creates a new runtime environment object which contains the runtime information for the encapsulated Nephele
@@ -177,7 +175,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
* @param invokableClass
* invokableClass the class that should be instantiated as a Nephele task
* @param taskConfiguration
- * the configuration object which was attached to the original {@link JobVertex}
+ * 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
@@ -240,60 +238,16 @@ public class RuntimeEnvironment implements Environment, Runnable {
this.invokable.setEnvironment(this);
this.invokable.registerInputOutput();
- if (!this.unboundOutputGateIDs.isEmpty() && LOG.isErrorEnabled()) {
- LOG.error("Inconsistency: " + this.unboundOutputGateIDs.size() + " unbound output gate IDs left");
- }
+ int numOutputGates = tdd.getNumberOfOutputGateDescriptors();
- if (!this.unboundInputGateIDs.isEmpty() && LOG.isErrorEnabled()) {
- LOG.error("Inconsistency: " + this.unboundInputGateIDs.size() + " unbound output gate IDs left");
+ for (int i = 0; i < numOutputGates; ++i) {
+ this.outputGates.get(i).initializeChannels(tdd.getOutputGateDescriptor(i));
}
- final int noogdd = tdd.getNumberOfOutputGateDescriptors();
- for (int i = 0; i < noogdd; ++i) {
- final GateDeploymentDescriptor gdd = tdd.getOutputGateDescriptor(i);
- final OutputGate og = this.outputGates.get(i);
- final ChannelType channelType = gdd.getChannelType();
- og.setChannelType(channelType);
+ int numInputGates = tdd.getNumberOfInputGateDescriptors();
- final int nocdd = gdd.getNumberOfChannelDescriptors();
- for (int j = 0; j < nocdd; ++j) {
-
- final ChannelDeploymentDescriptor cdd = gdd.getChannelDescriptor(j);
- switch (channelType) {
- case NETWORK:
- og.createNetworkOutputChannel(og, cdd.getOutputChannelID(), cdd.getInputChannelID());
- break;
- case INMEMORY:
- og.createInMemoryOutputChannel(og, cdd.getOutputChannelID(), cdd.getInputChannelID());
- break;
- default:
- throw new IllegalStateException("Unknown channel type");
- }
- }
- }
-
- final int noigdd = tdd.getNumberOfInputGateDescriptors();
- for (int i = 0; i < noigdd; ++i) {
- final GateDeploymentDescriptor gdd = tdd.getInputGateDescriptor(i);
- final InputGate ig = this.inputGates.get(i);
- final ChannelType channelType = gdd.getChannelType();
- ig.setChannelType(channelType);
-
- final int nicdd = gdd.getNumberOfChannelDescriptors();
- for (int j = 0; j < nicdd; ++j) {
-
- final ChannelDeploymentDescriptor cdd = gdd.getChannelDescriptor(j);
- switch (channelType) {
- case NETWORK:
- ig.createNetworkInputChannel(ig, cdd.getInputChannelID(), cdd.getOutputChannelID());
- break;
- case INMEMORY:
- ig.createInMemoryInputChannel(ig, cdd.getInputChannelID(), cdd.getOutputChannelID());
- break;
- default:
- throw new IllegalStateException("Unknown channel type");
- }
- }
+ for(int i = 0; i < numInputGates; i++){
+ this.inputGates.get(i).initializeChannels(tdd.getInputGateDescriptor(i));
}
}
@@ -306,29 +260,26 @@ public class RuntimeEnvironment implements Environment, Runnable {
return this.invokable;
}
-
@Override
public JobID getJobID() {
return this.jobID;
}
-
@Override
public GateID getNextUnboundInputGateID() {
-
return this.unboundInputGateIDs.poll();
}
+ @Override
+ public OutputGate createAndRegisterOutputGate() {
+ OutputGate gate = new OutputGate(getJobID(), new GateID(), getNumberOfOutputGates());
+ this.outputGates.add(gate);
- public GateID getNextUnboundOutputGateID() {
-
- return this.unboundOutputGateIDs.poll();
+ return gate;
}
-
@Override
public void run() {
-
if (invokable == null) {
LOG.fatal("ExecutionEnvironment has no Invokable set");
}
@@ -343,9 +294,6 @@ public class RuntimeEnvironment implements Environment, Runnable {
}
try {
-
- // Activate input channels
- // activateInputChannels();
ClassLoader cl = LibraryCacheManager.getClassLoader(jobID);
Thread.currentThread().setContextClassLoader(cl);
this.invokable.invoke();
@@ -354,9 +302,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
if (this.executionObserver.isCanceled()) {
throw new InterruptedException();
}
-
} catch (Throwable t) {
-
if (!this.executionObserver.isCanceled()) {
// Perform clean up when the task failed and has been not canceled by the user
@@ -370,7 +316,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
// Release all resources that may currently be allocated by the individual channels
releaseAllChannelResources();
- if (this.executionObserver.isCanceled()) {
+ if (this.executionObserver.isCanceled() || t instanceof CancelTaskException) {
changeExecutionState(ExecutionState.CANCELED, null);
} else {
changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(t));
@@ -399,7 +345,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
// Release all resources that may currently be allocated by the individual channels
releaseAllChannelResources();
- if (this.executionObserver.isCanceled()) {
+ if (this.executionObserver.isCanceled() || t instanceof CancelTaskException) {
changeExecutionState(ExecutionState.CANCELED, null);
} else {
changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(t));
@@ -415,62 +361,35 @@ public class RuntimeEnvironment implements Environment, Runnable {
changeExecutionState(ExecutionState.FINISHED, null);
}
-
@Override
- public <T extends IOReadableWritable> OutputGate<T> createOutputGate(final GateID gateID, Class<T> outputClass,
- final ChannelSelector<T> selector, final boolean isBroadcast) {
- final RuntimeOutputGate<T> rog = new RuntimeOutputGate<T>(getJobID(), gateID, outputClass,
- getNumberOfOutputGates(), selector, isBroadcast);
- return rog;
- }
+ public <T extends IOReadableWritable> InputGate<T> createAndRegisterInputGate() {
+ InputGate<T> gate = new InputGate<T>(getJobID(), new GateID(), getNumberOfInputGates());
+ this.inputGates.add(gate);
-
- @Override
- public <T extends IOReadableWritable> InputGate<T> createInputGate(final GateID gateID,
- final RecordDeserializerFactory<T> deserializerFactory) {
- final RuntimeInputGate<T> rig = new RuntimeInputGate<T>(getJobID(), gateID, deserializerFactory,
- getNumberOfInputGates());
- return rig;
- }
-
- @Override
- public void registerOutputGate(OutputGate<? extends IOReadableWritable> outputGate) {
-
- this.outputGates.add(outputGate);
+ return gate;
}
- @Override
- public void registerInputGate(InputGate<? extends IOReadableWritable> inputGate) {
- this.inputGates.add(inputGate);
- }
-
-
public int getNumberOfOutputGates() {
return this.outputGates.size();
}
-
@Override
public int getNumberOfInputGates() {
return this.inputGates.size();
}
-
@Override
public int getNumberOfOutputChannels() {
-
int numberOfOutputChannels = 0;
for (int i = 0; i < this.outputGates.size(); ++i) {
- numberOfOutputChannels += this.outputGates.get(i).getNumberOfOutputChannels();
+ numberOfOutputChannels += this.outputGates.get(i).getNumChannels();
}
return numberOfOutputChannels;
}
-
@Override
public int getNumberOfInputChannels() {
-
int numberOfInputChannels = 0;
for (int i = 0; i < this.inputGates.size(); ++i) {
numberOfInputChannels += this.inputGates.get(i).getNumberOfInputChannels();
@@ -497,13 +416,13 @@ public class RuntimeEnvironment implements Environment, Runnable {
/**
* Returns the registered output gate with index <code>pos</code>.
*
- * @param pos
+ * @param index
* 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
*/
- public OutputGate<? extends IOReadableWritable> getOutputGate(final int pos) {
- if (pos < this.outputGates.size()) {
- return this.outputGates.get(pos);
+ public OutputGate getOutputGate(int index) {
+ if (index < this.outputGates.size()) {
+ return this.outputGates.get(index);
}
return null;
@@ -515,7 +434,6 @@ public class RuntimeEnvironment implements Environment, Runnable {
* @return the thread which is assigned to execute the user code
*/
public Thread getExecutingThread() {
-
synchronized (this) {
if (this.executingThread == null) {
@@ -538,29 +456,14 @@ public class RuntimeEnvironment implements Environment, Runnable {
* @throws InterruptedException
* thrown if the thread waiting for the channels to be closed is interrupted
*/
- private void waitForOutputChannelsToBeClosed() throws IOException, InterruptedException {
-
- // Wait for disconnection of all output gates
- while (true) {
-
- // Make sure, we leave this method with an InterruptedException when the task has been canceled
- if (this.executionObserver.isCanceled()) {
- throw new InterruptedException();
- }
-
- boolean allClosed = true;
- for (int i = 0; i < getNumberOfOutputGates(); i++) {
- final OutputGate<? extends IOReadableWritable> og = this.outputGates.get(i);
- if (!og.isClosed()) {
- allClosed = false;
- }
- }
+ private void waitForOutputChannelsToBeClosed() throws InterruptedException {
+ // Make sure, we leave this method with an InterruptedException when the task has been canceled
+ if (this.executionObserver.isCanceled()) {
+ return;
+ }
- if (allClosed) {
- break;
- } else {
- Thread.sleep(SLEEPINTERVAL);
- }
+ for (OutputGate og : this.outputGates) {
+ og.waitForGateToBeClosed();
}
}
@@ -573,7 +476,6 @@ public class RuntimeEnvironment implements Environment, Runnable {
* thrown if the thread waiting for the channels to be closed is interrupted
*/
private void waitForInputChannelsToBeClosed() throws IOException, InterruptedException {
-
// Wait for disconnection of all output gates
while (true) {
@@ -602,7 +504,6 @@ public class RuntimeEnvironment implements Environment, Runnable {
* Closes all input gates which are not already closed.
*/
private void closeInputGates() throws IOException, InterruptedException {
-
for (int i = 0; i < this.inputGates.size(); i++) {
final InputGate<? extends IOReadableWritable> eig = this.inputGates.get(i);
// Important: close must be called on each input gate exactly once
@@ -615,61 +516,49 @@ public class RuntimeEnvironment implements Environment, Runnable {
* Requests all output gates to be closed.
*/
private void requestAllOutputGatesToClose() throws IOException, InterruptedException {
-
for (int i = 0; i < this.outputGates.size(); i++) {
this.outputGates.get(i).requestClose();
}
}
-
@Override
public IOManager getIOManager() {
return this.ioManager;
}
-
@Override
public MemoryManager getMemoryManager() {
return this.memoryManager;
}
-
@Override
public Configuration getTaskConfiguration() {
return this.taskConfiguration;
}
-
@Override
public Configuration getJobConfiguration() {
return this.jobConfiguration;
}
-
@Override
public int getCurrentNumberOfSubtasks() {
-
return this.currentNumberOfSubtasks;
}
-
@Override
public int getIndexInSubtaskGroup() {
-
return this.indexInSubtaskGroup;
}
private void changeExecutionState(final ExecutionState newExecutionState, final String optionalMessage) {
-
if (this.executionObserver != null) {
this.executionObserver.executionStateChanged(newExecutionState, optionalMessage);
}
}
-
@Override
public String getTaskName() {
-
return this.taskName;
}
@@ -679,9 +568,7 @@ public class RuntimeEnvironment implements Environment, Runnable {
* @return the name of the task with its index in the subtask group and the total number of subtasks
*/
public String getTaskNameWithIndex() {
-
- return this.taskName + " (" + (getIndexInSubtaskGroup() + 1) + "/"
- + getCurrentNumberOfSubtasks() + ")";
+ return String.format("%s (%d/%d)", this.taskName, getIndexInSubtaskGroup() + 1, getCurrentNumberOfSubtasks());
}
/**
@@ -694,25 +581,20 @@ public class RuntimeEnvironment implements Environment, Runnable {
this.executionObserver = executionObserver;
}
-
@Override
public InputSplitProvider getInputSplitProvider() {
return this.inputSplitProvider;
}
-
@Override
public void userThreadStarted(final Thread userThread) {
-
if (this.executionObserver != null) {
this.executionObserver.userThreadStarted(userThread);
}
}
-
@Override
public void userThreadFinished(final Thread userThread) {
-
if (this.executionObserver != null) {
this.executionObserver.userThreadFinished(userThread);
}
@@ -723,7 +605,6 @@ public class RuntimeEnvironment implements Environment, Runnable {
* method should only be called after the respected task has stopped running.
*/
private void releaseAllChannelResources() {
-
for (int i = 0; i < this.inputGates.size(); i++) {
this.inputGates.get(i).releaseAllChannelResources();
}
@@ -733,28 +614,21 @@ public class RuntimeEnvironment implements Environment, Runnable {
}
}
-
@Override
public Set<ChannelID> getOutputChannelIDs() {
+ Set<ChannelID> ids= new HashSet<ChannelID>();
- final Set<ChannelID> outputChannelIDs = new HashSet<ChannelID>();
-
- final Iterator<OutputGate<? extends IOReadableWritable>> gateIterator = this.outputGates.iterator();
- while (gateIterator.hasNext()) {
-
- final OutputGate<? extends IOReadableWritable> outputGate = gateIterator.next();
- for (int i = 0; i < outputGate.getNumberOfOutputChannels(); ++i) {
- outputChannelIDs.add(outputGate.getOutputChannel(i).getID());
+ for (OutputGate gate : this.outputGates) {
+ for (OutputChannel channel : gate.channels()) {
+ ids.add(channel.getID());
}
}
- return Collections.unmodifiableSet(outputChannelIDs);
+ return Collections.unmodifiableSet(ids);
}
-
@Override
public Set<ChannelID> getInputChannelIDs() {
-
final Set<ChannelID> inputChannelIDs = new HashSet<ChannelID>();
final Iterator<InputGate<? extends IOReadableWritable>> gateIterator = this.inputGates.iterator();
@@ -769,10 +643,8 @@ public class RuntimeEnvironment implements Environment, Runnable {
return Collections.unmodifiableSet(inputChannelIDs);
}
-
@Override
public Set<GateID> getInputGateIDs() {
-
final Set<GateID> inputGateIDs = new HashSet<GateID>();
final Iterator<InputGate<? extends IOReadableWritable>> gateIterator = this.inputGates.iterator();
@@ -783,13 +655,11 @@ public class RuntimeEnvironment implements Environment, Runnable {
return Collections.unmodifiableSet(inputGateIDs);
}
-
@Override
public Set<GateID> getOutputGateIDs() {
-
final Set<GateID> outputGateIDs = new HashSet<GateID>();
- final Iterator<OutputGate<? extends IOReadableWritable>> gateIterator = this.outputGates.iterator();
+ final Iterator<OutputGate> gateIterator = this.outputGates.iterator();
while (gateIterator.hasNext()) {
outputGateIDs.add(gateIterator.next().getGateID());
}
@@ -800,11 +670,10 @@ public class RuntimeEnvironment implements Environment, Runnable {
@Override
public Set<ChannelID> getOutputChannelIDsOfGate(final GateID gateID) {
-
- OutputGate<? extends IOReadableWritable> outputGate = null;
- final Iterator<OutputGate<? extends IOReadableWritable>> gateIterator = this.outputGates.iterator();
+ OutputGate outputGate = null;
+ final Iterator<OutputGate> gateIterator = this.outputGates.iterator();
while (gateIterator.hasNext()) {
- final OutputGate<? extends IOReadableWritable> candidateGate = gateIterator.next();
+ final OutputGate candidateGate = gateIterator.next();
if (candidateGate.getGateID().equals(gateID)) {
outputGate = candidateGate;
break;
@@ -817,8 +686,8 @@ public class RuntimeEnvironment implements Environment, Runnable {
final Set<ChannelID> outputChannelIDs = new HashSet<ChannelID>();
- for (int i = 0; i < outputGate.getNumberOfOutputChannels(); ++i) {
- outputChannelIDs.add(outputGate.getOutputChannel(i).getID());
+ for (int i = 0; i < outputGate.getNumChannels(); ++i) {
+ outputChannelIDs.add(outputGate.getChannel(i).getID());
}
return Collections.unmodifiableSet(outputChannelIDs);
@@ -827,7 +696,6 @@ public class RuntimeEnvironment implements Environment, Runnable {
@Override
public Set<ChannelID> getInputChannelIDsOfGate(final GateID gateID) {
-
InputGate<? extends IOReadableWritable> inputGate = null;
final Iterator<InputGate<? extends IOReadableWritable>> gateIterator = this.inputGates.iterator();
while (gateIterator.hasNext()) {
@@ -850,18 +718,86 @@ public class RuntimeEnvironment implements Environment, Runnable {
return Collections.unmodifiableSet(inputChannelIDs);
}
+
+ public List<OutputGate> outputGates() {
+ return this.outputGates;
+ }
+
+ public List<InputGate<? extends IOReadableWritable>> inputGates() {
+ return this.inputGates;
+ }
@Override
public AccumulatorProtocol getAccumulatorProtocolProxy() {
return accumulatorProtocolProxy;
}
- public void addCopyTaskForCacheFile(String name, FutureTask<Path> copyTask) {
- this.cacheCopyTasks.put(name, copyTask);
+ @Override
+ public BufferProvider getOutputBufferProvider() {
+ return this;
}
+
+ // -----------------------------------------------------------------------------------------------------------------
+ // BufferProvider methods
+ // -----------------------------------------------------------------------------------------------------------------
+
@Override
- public Map<String, FutureTask<Path>> getCopyTask() {
- return this.cacheCopyTasks;
+ public Buffer requestBuffer(int minBufferSize) throws IOException {
+ return this.outputBufferPool.requestBuffer(minBufferSize);
}
+
+ @Override
+ public Buffer requestBufferBlocking(int minBufferSize) throws IOException, InterruptedException {
+ return this.outputBufferPool.requestBufferBlocking(minBufferSize);
+ }
+
+ @Override
+ public int getBufferSize() {
+ return this.outputBufferPool.getBufferSize();
+ }
+
+ @Override
+ public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+ return this.outputBufferPool.registerBufferAvailabilityListener(listener);
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+ // LocalBufferPoolOwner methods
+ // -----------------------------------------------------------------------------------------------------------------
+ @Override
+ public int getNumberOfChannels() {
+ return getNumberOfOutputChannels();
+ }
+
+ @Override
+ public void setDesignatedNumberOfBuffers(int numBuffers) {
+ this.outputBufferPool.setNumDesignatedBuffers(numBuffers);
+ }
+
+ @Override
+ public void clearLocalBufferPool() {
+ this.outputBufferPool.destroy();
+ }
+
+ @Override
+ public void registerGlobalBufferPool(GlobalBufferPool globalBufferPool) {
+ if (this.outputBufferPool == null) {
+ this.outputBufferPool = new LocalBufferPool(globalBufferPool, 1);
+ }
+ }
+
+ @Override
+ public void logBufferUtilization() {
+ LOG.info(String.format("\t%s: %d available, %d requested, %d designated",
+ getTaskNameWithIndex(),
+ this.outputBufferPool.numAvailableBuffers(),
+ this.outputBufferPool.numRequestedBuffers(),
+ this.outputBufferPool.numDesignatedBuffers()));
+ }
+
+ @Override
+ public void reportAsynchronousEvent() {
+ this.outputBufferPool.reportAsynchronousEvent();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/DistributionPatternProvider.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/DistributionPatternProvider.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/DistributionPatternProvider.java
index 0c004ad..35350cc 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/DistributionPatternProvider.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/DistributionPatternProvider.java
@@ -13,7 +13,7 @@
package eu.stratosphere.nephele.executiongraph;
-import eu.stratosphere.nephele.io.DistributionPattern;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
public final class DistributionPatternProvider {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionEdge.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionEdge.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionEdge.java
index cbb957f..0106361 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionEdge.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionEdge.java
@@ -13,11 +13,11 @@
package eu.stratosphere.nephele.executiongraph;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelType;
/**
- * Objects of this class represent a pair of {@link AbstractInputChannel} and {@link AbstractOutputChannel} objects
+ * Objects of this class represent a pair of {@link eu.stratosphere.runtime.io.serialization.io.channels.InputChannel} and {@link AbstractOutputChannel} objects
* within an {@link ExecutionGraph}, Nephele's internal scheduling representation for jobs.
*
*/
@@ -85,11 +85,6 @@ public final class ExecutionEdge {
return this.groupEdge.getChannelType();
}
- public boolean isBroadcast() {
-
- return this.groupEdge.isBroadcast();
- }
-
public int getConnectionID() {
return this.groupEdge.getConnectionID();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGate.java
index f1bcb76..20d1a4e 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGate.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGate.java
@@ -16,8 +16,8 @@ package eu.stratosphere.nephele.executiongraph;
import java.util.Collection;
import java.util.concurrent.CopyOnWriteArrayList;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.gates.GateID;
+import eu.stratosphere.runtime.io.channels.ChannelType;
/**
* Objects of this class represent either an {@link InputGate} or {@link OutputGate} within an {@link ExecutionGraph},
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
index aa08655..ca7eddb 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGraph.java
@@ -40,17 +40,17 @@ import eu.stratosphere.nephele.instance.AllocatedResource;
import eu.stratosphere.nephele.instance.DummyInstance;
import eu.stratosphere.nephele.instance.InstanceManager;
import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.gates.GateID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.nephele.jobgraph.AbstractJobInputVertex;
import eu.stratosphere.nephele.jobgraph.AbstractJobVertex;
import eu.stratosphere.nephele.jobgraph.JobEdge;
import eu.stratosphere.nephele.jobgraph.JobFileOutputVertex;
import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.runtime.ExecutorThreadFactory;
+import eu.stratosphere.nephele.taskmanager.ExecutorThreadFactory;
import eu.stratosphere.nephele.template.AbstractInputTask;
import eu.stratosphere.nephele.template.AbstractInvokable;
import eu.stratosphere.util.StringUtils;
@@ -412,9 +412,6 @@ public class ExecutionGraph implements ExecutionListener {
// First, build the group edges
for (int i = 0; i < sjv.getNumberOfForwardConnections(); ++i) {
-
- final boolean isBroadcast = sgv.getEnvironment().getOutputGate(i).isBroadcast();
-
final JobEdge edge = sjv.getForwardConnection(i);
final AbstractJobVertex tjv = edge.getConnectedVertex();
@@ -427,13 +424,12 @@ public class ExecutionGraph implements ExecutionListener {
userDefinedChannelType = false;
channelType = ChannelType.NETWORK;
}
- // Use NO_COMPRESSION as default compression level if nothing else is defined by the user
final DistributionPattern distributionPattern = edge.getDistributionPattern();
// Connect the corresponding group vertices and copy the user settings from the job edge
final ExecutionGroupEdge groupEdge = sgv.wireTo(tgv, edge.getIndexOfInputGate(), i, channelType,
- userDefinedChannelType,distributionPattern, isBroadcast);
+ userDefinedChannelType,distributionPattern);
final ExecutionGate outputGate = new ExecutionGate(new GateID(), sev, groupEdge, false);
sev.insertOutputGate(i, outputGate);
@@ -970,7 +966,7 @@ public class ExecutionGraph implements ExecutionListener {
final ExecutionGate outputGate = sourceVertex.getOutputGate(i);
final ChannelType channelType = outputGate.getChannelType();
- if (channelType == ChannelType.INMEMORY) {
+ if (channelType == ChannelType.IN_MEMORY) {
final int numberOfOutputChannels = outputGate.getNumberOfEdges();
for (int j = 0; j < numberOfOutputChannels; ++j) {
final ExecutionEdge outputChannel = outputGate.getEdge(j);
@@ -990,7 +986,7 @@ public class ExecutionGraph implements ExecutionListener {
final ExecutionGate inputGate = targetVertex.getInputGate(i);
final ChannelType channelType = inputGate.getChannelType();
- if (channelType == ChannelType.INMEMORY) {
+ if (channelType == ChannelType.IN_MEMORY) {
final int numberOfInputChannels = inputGate.getNumberOfEdges();
for (int j = 0; j < numberOfInputChannels; ++j) {
final ExecutionEdge inputChannel = inputGate.getEdge(j);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupEdge.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupEdge.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupEdge.java
index 14970fd..373d120 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupEdge.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupEdge.java
@@ -13,8 +13,8 @@
package eu.stratosphere.nephele.executiongraph;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
/**
* An execution group edge represents an edge between two execution group vertices.
@@ -66,11 +66,6 @@ public class ExecutionGroupEdge {
private final DistributionPattern distributionPattern;
/**
- * Stores if the edge is part of a broadcast group.
- */
- private final boolean isBroadcast;
-
- /**
* Constructs a new group edge.
*
* @param sourceVertex
@@ -85,19 +80,12 @@ public class ExecutionGroupEdge {
* the channel type for the edge
* @param userDefinedChannelType
* <code>true</code> if the channel type has been specified by the user, <code>false</code> otherwise
- * @param compressionLevel
- * the compression level for the edge
- * @param userDefinedCompressionLevel
- * <code>true</code> if the compression level has been specified by the user, <code>false</code> otherwise
* @param distributionPattern
* the distribution pattern to create the wiring
- * @param isBroadcast
- * indicates that the edge is part of a broadcast group
*/
public ExecutionGroupEdge(final ExecutionGroupVertex sourceVertex, final int indexOfOutputGate,
final ExecutionGroupVertex targetVertex, final int indexOfInputGate, final ChannelType channelType,
- final boolean userDefinedChannelType, final DistributionPattern distributionPattern,
- final boolean isBroadcast) {
+ final boolean userDefinedChannelType, final DistributionPattern distributionPattern) {
this.sourceVertex = sourceVertex;
this.indexOfOutputGate = indexOfOutputGate;
this.channelType = channelType;
@@ -105,7 +93,6 @@ public class ExecutionGroupEdge {
this.userDefinedChannelType = userDefinedChannelType;
this.targetVertex = targetVertex;
this.distributionPattern = distributionPattern;
- this.isBroadcast = isBroadcast;
}
/**
@@ -206,13 +193,4 @@ public class ExecutionGroupEdge {
public DistributionPattern getDistributionPattern() {
return this.distributionPattern;
}
-
- /**
- * Checks if the edge is part of a broadcast group.
- *
- * @return <code>true</code> if the edge is part of a broadcast group, <code>false</code> otherwise
- */
- public boolean isBroadcast() {
- return this.isBroadcast;
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
index 908eb6e..89b4b6d 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionGroupVertex.java
@@ -13,6 +13,17 @@
package eu.stratosphere.nephele.executiongraph;
+import eu.stratosphere.configuration.Configuration;
+import eu.stratosphere.core.io.InputSplit;
+import eu.stratosphere.nephele.execution.RuntimeEnvironment;
+import eu.stratosphere.nephele.instance.AllocatedResource;
+import eu.stratosphere.nephele.instance.DummyInstance;
+import eu.stratosphere.nephele.instance.InstanceType;
+import eu.stratosphere.nephele.jobgraph.JobVertexID;
+import eu.stratosphere.nephele.template.AbstractInvokable;
+import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
@@ -23,17 +34,6 @@ import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.core.io.InputSplit;
-import eu.stratosphere.nephele.execution.RuntimeEnvironment;
-import eu.stratosphere.nephele.instance.AllocatedResource;
-import eu.stratosphere.nephele.instance.DummyInstance;
-import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.jobgraph.JobVertexID;
-import eu.stratosphere.nephele.template.AbstractInvokable;
-
/**
* An ExecutionGroupVertex is created for every JobVertex of the initial job graph. It represents a number of execution
* vertices that originate from the same job vertex.
@@ -388,7 +388,7 @@ public final class ExecutionGroupVertex {
*/
ExecutionGroupEdge wireTo(final ExecutionGroupVertex groupVertex, final int indexOfInputGate,
final int indexOfOutputGate, final ChannelType channelType, final boolean userDefinedChannelType,
- final DistributionPattern distributionPattern, final boolean isBroadcast) throws GraphConversionException {
+ final DistributionPattern distributionPattern) throws GraphConversionException {
try {
final ExecutionGroupEdge previousEdge = this.forwardLinks.get(indexOfOutputGate);
@@ -401,8 +401,7 @@ public final class ExecutionGroupVertex {
}
final ExecutionGroupEdge edge = new ExecutionGroupEdge(this, indexOfOutputGate, groupVertex, indexOfInputGate,
- channelType, userDefinedChannelType, distributionPattern,
- isBroadcast);
+ channelType, userDefinedChannelType, distributionPattern);
this.forwardLinks.add(edge);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java
index a251d3c..eab2375 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionStage.java
@@ -27,7 +27,7 @@ import eu.stratosphere.nephele.instance.AbstractInstance;
import eu.stratosphere.nephele.instance.DummyInstance;
import eu.stratosphere.nephele.instance.InstanceRequestMap;
import eu.stratosphere.nephele.instance.InstanceType;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
/**
* An execution stage contains all execution group vertices (and as a result all execution vertices) which
@@ -401,7 +401,7 @@ public final class ExecutionStage {
recurse = true;
}
- if (channelType == ChannelType.INMEMORY
+ if (channelType == ChannelType.IN_MEMORY
&& !pipeline.equals(connectedVertex.getExecutionPipeline())) {
connectedVertex.setExecutionPipeline(pipeline);
@@ -432,7 +432,7 @@ public final class ExecutionStage {
recurse = true;
}
- if (channelType == ChannelType.INMEMORY
+ if (channelType == ChannelType.IN_MEMORY
&& !pipeline.equals(connectedVertex.getExecutionPipeline())) {
connectedVertex.setExecutionPipeline(pipeline);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java
index 7a9c48b..57ff073 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java
@@ -35,11 +35,10 @@ import eu.stratosphere.nephele.execution.ExecutionState;
import eu.stratosphere.nephele.execution.ExecutionStateTransition;
import eu.stratosphere.nephele.instance.AllocatedResource;
import eu.stratosphere.nephele.instance.AllocationID;
-import eu.stratosphere.nephele.io.GateID;
+import eu.stratosphere.runtime.io.gates.GateID;
import eu.stratosphere.nephele.taskmanager.AbstractTaskResult;
import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode;
import eu.stratosphere.nephele.taskmanager.TaskCancelResult;
-import eu.stratosphere.nephele.taskmanager.TaskKillResult;
import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult;
import eu.stratosphere.nephele.util.AtomicEnum;
import eu.stratosphere.nephele.util.SerializableArrayList;
@@ -692,41 +691,6 @@ 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
* from the instance it is currently running on. If the task
* is not currently running, its execution state is simply
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertexID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertexID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertexID.java
index 266b6ef..6209fde 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertexID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertexID.java
@@ -13,7 +13,7 @@
package eu.stratosphere.nephele.executiongraph;
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
import eu.stratosphere.nephele.managementgraph.ManagementVertexID;
/**
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java
index cd768e2..72e3651 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ManagementGraphFactory.java
@@ -18,7 +18,7 @@ import java.util.Iterator;
import java.util.Map;
import eu.stratosphere.nephele.instance.AbstractInstance;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.nephele.managementgraph.ManagementEdge;
import eu.stratosphere.nephele.managementgraph.ManagementEdgeID;
import eu.stratosphere.nephele.managementgraph.ManagementGate;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java
index 898f8e9..50e0e7f 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java
@@ -24,13 +24,12 @@ import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse;
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate;
import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
-import eu.stratosphere.nephele.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.nephele.ipc.RPC;
import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.nephele.net.NetUtils;
import eu.stratosphere.nephele.protocols.TaskOperationProtocol;
import eu.stratosphere.nephele.taskmanager.TaskCancelResult;
-import eu.stratosphere.nephele.taskmanager.TaskKillResult;
import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult;
import eu.stratosphere.nephele.topology.NetworkNode;
import eu.stratosphere.nephele.topology.NetworkTopology;
@@ -96,8 +95,8 @@ public abstract class AbstractInstance extends NetworkNode {
if (this.taskManager == null) {
this.taskManager = RPC.getProxy(TaskOperationProtocol.class,
- new InetSocketAddress(getInstanceConnectionInfo().getAddress(),
- getInstanceConnectionInfo().getIPCPort()), NetUtils.getSocketFactory());
+ new InetSocketAddress(getInstanceConnectionInfo().address(),
+ getInstanceConnectionInfo().ipcPort()), NetUtils.getSocketFactory());
}
return this.taskManager;
@@ -206,22 +205,6 @@ public abstract class AbstractInstance extends NetworkNode {
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
public boolean equals(final Object obj) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocationID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocationID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocationID.java
index 7e8339f..3c83b80 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocationID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AllocationID.java
@@ -13,7 +13,7 @@
package eu.stratosphere.nephele.instance;
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
/**
* An allocation ID unambiguously identifies the allocated resources
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java
index a0fd608..257421b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java
@@ -154,7 +154,8 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
*
* @return the port instance's task manager runs its IPC service on
*/
- public int getIPCPort() {
+ public int ipcPort() {
+
return this.ipcPort;
}
@@ -163,7 +164,8 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
*
* @return the port instance's task manager expects to receive transfer envelopes on
*/
- public int getDataPort() {
+ public int dataPort() {
+
return this.dataPort;
}
@@ -172,7 +174,8 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
*
* @return the network address the instance's task manager binds its sockets to
*/
- public InetAddress getAddress() {
+ public InetAddress address() {
+
return this.inetAddress;
}
@@ -182,7 +185,8 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
*
* @return the host name of the instance
*/
- public String getHostName() {
+ public String hostname() {
+
return this.hostName;
}
@@ -191,7 +195,8 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
*
* @return the domain name of the instance or <code>null</code> if the domain name could not be determined
*/
- public String getDomainName() {
+ public String domainName() {
+
return this.domainName;
}
@@ -251,15 +256,15 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
if (obj instanceof InstanceConnectionInfo) {
InstanceConnectionInfo ici = (InstanceConnectionInfo) obj;
- if (!this.inetAddress.equals(ici.getAddress())) {
+ if (!this.inetAddress.equals(ici.address())) {
return false;
}
- if (this.ipcPort != ici.getIPCPort()) {
+ if (this.ipcPort != ici.ipcPort()) {
return false;
}
- if (this.dataPort != ici.getDataPort()) {
+ if (this.dataPort != ici.dataPort()) {
return false;
}
@@ -278,7 +283,9 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
@Override
public int compareTo(final InstanceConnectionInfo o) {
- return this.getAddress().getHostName().compareTo(((InstanceConnectionInfo) o).getAddress().getHostName());
+
+ return this.address().getHostName()
+ .compareTo(((InstanceConnectionInfo) o).address().getHostName());
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceID.java
index c93b6bd..9b5b707 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceID.java
@@ -13,7 +13,7 @@
package eu.stratosphere.nephele.instance;
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
/**
* Class for statistically unique instance IDs.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterManager.java
index b69d23e..480e521 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/cluster/ClusterManager.java
@@ -396,14 +396,14 @@ public class ClusterManager implements InstanceManager {
final HardwareDescription hardwareDescription) {
// Check if there is a user-defined instance type for this IP address
- InstanceType instanceType = this.ipToInstanceTypeMapping.get(instanceConnectionInfo.getAddress());
+ InstanceType instanceType = this.ipToInstanceTypeMapping.get(instanceConnectionInfo.address());
if (instanceType != null) {
LOG.info("Found user-defined instance type for cluster instance with IP "
- + instanceConnectionInfo.getAddress() + ": " + instanceType);
+ + instanceConnectionInfo.address() + ": " + instanceType);
} else {
instanceType = matchHardwareDescriptionWithInstanceType(hardwareDescription);
if (instanceType != null) {
- LOG.info("Hardware profile of cluster instance with IP " + instanceConnectionInfo.getAddress()
+ LOG.info("Hardware profile of cluster instance with IP " + instanceConnectionInfo.address()
+ " matches with instance type " + instanceType);
} else {
LOG.error("No matching instance type, cannot create cluster instance");
@@ -412,7 +412,7 @@ public class ClusterManager implements InstanceManager {
}
// Try to match new host with a stub host from the existing topology
- String instanceName = instanceConnectionInfo.getHostName();
+ String instanceName = instanceConnectionInfo.hostname();
NetworkNode parentNode = this.networkTopology.getRootNode();
NetworkNode currentStubNode = null;
@@ -439,7 +439,7 @@ public class ClusterManager implements InstanceManager {
// Try to match the new host using the IP address
if (currentStubNode == null) {
- instanceName = instanceConnectionInfo.getAddress().toString();
+ instanceName = instanceConnectionInfo.address().toString();
instanceName = instanceName.replaceAll("/", ""); // Remove any / characters
currentStubNode = this.networkTopology.getNodeByName(instanceName);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java
deleted file mode 100644
index c3e3697..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractGate.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.event.task.EventListener;
-import eu.stratosphere.nephele.event.task.EventNotificationManager;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * In Nephele a gate represents the connection between a user program and the processing framework. A gate
- * must be connected to exactly one record reader/writer and to at least one channel. The <code>Gate</code> class itself
- * is abstract. A gate automatically created for every record reader/writer in the user program. A gate can only be used
- * to transport one specific type of records.
- * <p>
- * This class in general is not thread-safe.
- *
- * @param <T>
- * the record type to be transported from this gate
- */
-public abstract class AbstractGate<T extends IOReadableWritable> implements Gate<T> {
-
- /**
- * The ID of the job this gate belongs to.
- */
- private final JobID jobID;
-
- /**
- * The ID of this gate.
- */
- private final GateID gateID;
-
- /**
- * The index of the gate in the list of available input/output gates.
- */
- private final int index;
-
- /**
- * The event notification manager used to dispatch events.
- */
- private final EventNotificationManager eventNotificationManager = new EventNotificationManager();
-
- /**
- * The type of input/output channels connected to this gate.
- */
- private ChannelType channelType = ChannelType.NETWORK;
-
- /**
- * Constructs a new abstract gate
- *
- * @param jobID
- * the ID of the job this gate belongs to
- * @param gateID
- * the ID of this gate
- * @param index
- * the index of the gate in the list of available input/output gates.
- */
- protected AbstractGate(final JobID jobID, final GateID gateID, final int index) {
- this.jobID = jobID;
- this.gateID = gateID;
- this.index = index;
- }
-
-
- @Override
- public final int getIndex() {
- return this.index;
- }
-
- /**
- * Returns the event notification manager used to dispatch events.
- *
- * @return the event notification manager used to dispatch events
- */
- protected final EventNotificationManager getEventNotificationManager() {
- return this.eventNotificationManager;
- }
-
-
- @Override
- public String toString() {
-
- return "Gate " + this.index;
- }
-
-
- @Override
- public final void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
-
- this.eventNotificationManager.subscribeToEvent(eventListener, eventType);
- }
-
-
- @Override
- public final void unsubscribeFromEvent(final EventListener eventListener,
- final Class<? extends AbstractTaskEvent> eventType) {
-
- this.eventNotificationManager.unsubscribeFromEvent(eventListener, eventType);
- }
-
-
- @Override
- public final void deliverEvent(final AbstractTaskEvent event) {
-
- this.eventNotificationManager.deliverEvent((AbstractTaskEvent) event);
- }
-
-
- @Override
- public final void setChannelType(final ChannelType channelType) {
-
- this.channelType = channelType;
- }
-
-
- @Override
- public final ChannelType getChannelType() {
-
- return this.channelType;
- }
-
-
- @Override
- public JobID getJobID() {
-
- return this.jobID;
- }
-
-
- @Override
- public GateID getGateID() {
-
- return this.gateID;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractID.java
deleted file mode 100644
index 37e6cbb..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractID.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.util.StringUtils;
-
-/**
- * ID is an abstract base class for providing statistically unique identification numbers in Nephele.
- * Every component that requires these kinds of IDs provides its own concrete type.
- * <p>
- * This class is thread-safe.
- *
- */
-public abstract class AbstractID implements IOReadableWritable {
-
- /**
- * The size of a long in bytes.
- */
- private static final int SIZE_OF_LONG = 8;
-
- /**
- * The size of the ID in byte.
- */
- protected static final int SIZE = 2 * SIZE_OF_LONG;
-
- /**
- * The upper part of the actual ID.
- */
- private long upperPart;
-
- /**
- * The lower part of the actual ID.
- */
- private long lowerPart;
-
- /**
- * Constructs a new ID with a specific bytes value.
- */
- public AbstractID(final byte[] bytes) {
-
- if (bytes.length != SIZE) {
- throw new IllegalArgumentException("Argument bytes must by an array of " + SIZE + " bytes");
- }
-
- this.lowerPart = byteArrayToLong(bytes, 0);
- this.upperPart = byteArrayToLong(bytes, SIZE_OF_LONG);
- }
-
- /**
- * Constructs a new abstract ID.
- *
- * @param lowerPart
- * the lower bytes of the ID
- * @param upperPart
- * the higher bytes of the ID
- */
- protected AbstractID(final long lowerPart, final long upperPart) {
-
- this.lowerPart = lowerPart;
- this.upperPart = upperPart;
- }
-
- /**
- * Creates a new abstract ID from the given one. The given and the newly created abtract ID will be identical, i.e.
- * a comparison by <code>equals</code> will return <code>true</code> and both objects will have the same hash code.
- *
- * @param id
- * the abstract ID to copy
- */
- protected AbstractID(final AbstractID id) {
-
- this.lowerPart = id.lowerPart;
- this.upperPart = id.upperPart;
- }
-
- /**
- * Constructs a new random ID from a uniform distribution.
- */
- public AbstractID() {
-
- this.lowerPart = generateRandomBytes();
- this.upperPart = generateRandomBytes();
- }
-
- /**
- * Generates a uniformly distributed random positive long.
- *
- * @return a uniformly distributed random positive long
- */
- protected static long generateRandomBytes() {
-
- return (long) (Math.random() * Long.MAX_VALUE);
- }
-
- /**
- * Converts the given byte array to a long.
- *
- * @param ba
- * the byte array to be converted
- * @param offset
- * the offset indicating at which byte inside the array the conversion shall begin
- * @return the long variable
- */
- private static long byteArrayToLong(final byte[] ba, final int offset) {
-
- long l = 0;
-
- for (int i = 0; i < SIZE_OF_LONG; ++i) {
- l |= (ba[offset + SIZE_OF_LONG - 1 - i] & 0xffL) << (i << 3);
- }
-
- return l;
- }
-
- /**
- * Converts a long to a byte array.
- *
- * @param l
- * the long variable to be converted
- * @param ba
- * the byte array to store the result the of the conversion
- * @param offset
- * the offset indicating at what position inside the byte array the result of the conversion shall be stored
- */
- private static void longToByteArray(final long l, final byte[] ba, final int offset) {
-
- for (int i = 0; i < SIZE_OF_LONG; ++i) {
- final int shift = i << 3; // i * 8
- ba[offset + SIZE_OF_LONG - 1 - i] = (byte) ((l & (0xffL << shift)) >>> shift);
- }
- }
-
- /**
- * Sets an ID from another ID by copying its internal byte representation.
- *
- * @param src
- * the source ID
- */
- public void setID(final AbstractID src) {
- this.lowerPart = src.lowerPart;
- this.upperPart = src.upperPart;
- }
-
-
- @Override
- public boolean equals(final Object obj) {
-
- if (!(obj instanceof AbstractID)) {
- return false;
- }
-
- final AbstractID src = (AbstractID) obj;
-
- if (src.lowerPart != this.lowerPart) {
- return false;
- }
-
- if (src.upperPart != this.upperPart) {
- return false;
- }
-
- return true;
- }
-
-
- @Override
- public int hashCode() {
-
- return (int) (this.lowerPart ^ (this.upperPart >>> 32));
- }
-
-
- @Override
- public void read(final DataInput in) throws IOException {
-
- this.lowerPart = in.readLong();
- this.upperPart = in.readLong();
- }
-
-
- @Override
- public void write(final DataOutput out) throws IOException {
-
- out.writeLong(this.lowerPart);
- out.writeLong(this.upperPart);
- }
-
-
- @Override
- public String toString() {
-
- final byte[] ba = new byte[SIZE];
- longToByteArray(this.lowerPart, ba, 0);
- longToByteArray(this.upperPart, ba, SIZE_OF_LONG);
-
- return StringUtils.byteToHexString(ba);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java
deleted file mode 100644
index 3a8ed25..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordReader.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.event.task.EventListener;
-import eu.stratosphere.nephele.event.task.EventNotificationManager;
-
-/**
- * This is an abstract base class for a record reader, either dealing with mutable or immutable records,
- * and dealing with reads from single gates (single end points) or multiple gates (union).
- */
-public abstract class AbstractRecordReader implements ReaderBase {
-
-
- private final EventNotificationManager eventHandler = new EventNotificationManager();
-
- private int numEventsUntilEndOfSuperstep = -1;
-
- private int endOfSuperstepEventsCount;
-
- // --------------------------------------------------------------------------------------------
-
- /**
- * Subscribes the listener object to receive events of the given type.
- *
- * @param eventListener
- * the listener object to register
- * @param eventType
- * the type of event to register the listener for
- */
- @Override
- public void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
- this.eventHandler.subscribeToEvent(eventListener, eventType);
- }
-
- /**
- * Removes the subscription for events of the given type for the listener object.
- *
- * @param eventListener The listener object to cancel the subscription for.
- * @param eventType The type of the event to cancel the subscription for.
- */
- @Override
- public void unsubscribeFromEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
- this.eventHandler.unsubscribeFromEvent(eventListener, eventType);
- }
-
-
- protected void handleEvent(AbstractTaskEvent evt) {
- this.eventHandler.deliverEvent(evt);
- }
-
- @Override
- public void setIterative(int numEventsUntilEndOfSuperstep) {
- this.numEventsUntilEndOfSuperstep = numEventsUntilEndOfSuperstep;
- }
-
- @Override
- public void startNextSuperstep() {
- if (this.numEventsUntilEndOfSuperstep == -1) {
- throw new IllegalStateException("Called 'startNextSuperstep()' in a non-iterative reader.");
- }
- else if (endOfSuperstepEventsCount < numEventsUntilEndOfSuperstep) {
- throw new IllegalStateException("Premature 'startNextSuperstep()'. Not yet reached the end-of-superstep.");
- }
- this.endOfSuperstepEventsCount = 0;
- }
-
- @Override
- public boolean hasReachedEndOfSuperstep() {
- return endOfSuperstepEventsCount== numEventsUntilEndOfSuperstep;
- }
-
- protected boolean incrementEndOfSuperstepEventAndCheck() {
- if (numEventsUntilEndOfSuperstep == -1) {
- throw new IllegalStateException("Received EndOfSuperstep event in a non-iterative reader.");
- }
-
- endOfSuperstepEventsCount++;
-
- if (endOfSuperstepEventsCount > numEventsUntilEndOfSuperstep) {
- throw new IllegalStateException("Received EndOfSuperstep events beyond the number to indicate the end of the superstep");
- }
-
- return endOfSuperstepEventsCount== numEventsUntilEndOfSuperstep;
- }
-}
[16/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/NoBufferAvailableException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/NoBufferAvailableException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/NoBufferAvailableException.java
deleted file mode 100644
index fe25837..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/NoBufferAvailableException.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/***********************************************************************************************************************
- * 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.transferenvelope;
-
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-
-/**
- * This exception is thrown to indicate that the deserialization process of a {@link TransferEnvelope} could not be
- * continued because a {@link Buffer} to store the envelope's content is currently not available.
- *
- */
-public final class NoBufferAvailableException extends Exception {
-
- /**
- * Generated serial UID.
- */
- 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
- * the buffer provider which could not deliver a buffer
- */
- NoBufferAvailableException(final BufferProvider bufferProvider) {
- this.bufferProvider = bufferProvider;
- }
-
- /**
- * Returns the buffer provider which could not deliver a buffer.
- *
- * @return the buffer provider which could not deliver a buffer
- */
- public BufferProvider getBufferProvider() {
- return this.bufferProvider;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelope.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelope.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelope.java
deleted file mode 100644
index 32c56c5..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelope.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/***********************************************************************************************************************
- * 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.transferenvelope;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.EventList;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-public final class TransferEnvelope {
-
- private final JobID jobID;
-
- private final ChannelID source;
-
- private final int sequenceNumber;
-
- private EventList eventList;
-
- private Buffer buffer = null;
-
- public TransferEnvelope(int sequenceNumber, JobID jobID, ChannelID source) {
- this(sequenceNumber, jobID, source, null);
- }
-
- public TransferEnvelope(int sequenceNumber, JobID jobID, ChannelID source, EventList eventList) {
-
- this.sequenceNumber = sequenceNumber;
- this.jobID = jobID;
- this.source = source;
- this.eventList = eventList;
- }
-
- public JobID getJobID() {
- return this.jobID;
- }
-
- public ChannelID getSource() {
- return this.source;
- }
-
- public void addEvent(AbstractEvent event) {
-
- if (this.eventList == null) {
- this.eventList = new EventList();
- }
-
- this.eventList.add(event);
- }
-
- public EventList getEventList() {
-
- return this.eventList;
- }
-
- public int getSequenceNumber() {
- return this.sequenceNumber;
- }
-
- public void setBuffer(Buffer buffer) {
- this.buffer = buffer;
- }
-
- public Buffer getBuffer() {
- return this.buffer;
- }
-
- public TransferEnvelope duplicate() throws IOException, InterruptedException {
-
- final TransferEnvelope duplicatedTransferEnvelope = new TransferEnvelope(this.sequenceNumber, this.jobID,
- this.source, this.eventList); // No need to duplicate event list
-
- if (this.buffer != null) {
- duplicatedTransferEnvelope.buffer = this.buffer.duplicate();
- } else {
- duplicatedTransferEnvelope.buffer = null;
- }
-
- return duplicatedTransferEnvelope;
- }
-
- public TransferEnvelope duplicateWithoutBuffer() {
-
- final TransferEnvelope duplicatedTransferEnvelope = new TransferEnvelope(this.sequenceNumber, this.jobID,
- this.source, this.eventList); // No need to duplicate event list
-
- duplicatedTransferEnvelope.buffer = null;
-
- return duplicatedTransferEnvelope;
- }
-
- @Override
- public boolean equals(final Object obj) {
-
- if (!(obj instanceof TransferEnvelope)) {
- return false;
- }
-
- final TransferEnvelope te = (TransferEnvelope) obj;
-
- if (!this.jobID.equals(te.jobID)) {
- return false;
- }
-
- if (!this.source.equals(te.source)) {
- return false;
- }
-
- if (this.sequenceNumber != te.sequenceNumber) {
- return false;
- }
-
- if (this.buffer == null) {
- if (te.buffer != null) {
- return false;
- }
- // Both are null
- } else {
- if (te.buffer == null) {
- return false;
- }
- // Both are non-null
- if (!this.buffer.equals(te.buffer)) {
- return false;
- }
- }
-
- if (this.eventList == null) {
- if (te.eventList != null) {
- return false;
- }
- // Both are null
- } else {
- if (te.eventList == null) {
- return false;
- }
- // Both are non-null
- if (!this.eventList.equals(te.eventList)) {
- return false;
- }
- }
-
- return true;
- }
-
- @Override
- public int hashCode() {
-
- return (31 * this.sequenceNumber * this.jobID.hashCode());
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeDispatcher.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeDispatcher.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeDispatcher.java
deleted file mode 100644
index 7e1fb87..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeDispatcher.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/***********************************************************************************************************************
- * 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.transferenvelope;
-
-import java.io.IOException;
-
-/**
- * A transfer envelope dispatcher receives {@link TransferEnvelopes} and sends them to all of this destinations.
- *
- */
-public interface TransferEnvelopeDispatcher {
-
- /**
- * Processes a transfer envelope from an output channel. The method may block until the system has allocated enough
- * resources to further process the envelope.
- *
- * @param transferEnvelope
- * the transfer envelope to be processed
- */
- void processEnvelopeFromOutputChannel(TransferEnvelope transferEnvelope) throws IOException, InterruptedException;
-
- void processEnvelopeFromInputChannel(TransferEnvelope transferEnvelope) throws IOException, InterruptedException;
-
- void processEnvelopeFromNetwork(TransferEnvelope transferEnvelope, boolean freeSourceBuffer) throws IOException,
- InterruptedException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeReceiverList.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeReceiverList.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeReceiverList.java
deleted file mode 100644
index 678897a..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/TransferEnvelopeReceiverList.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/***********************************************************************************************************************
- * 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.transferenvelope;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ConnectionInfoLookupResponse;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.RemoteReceiver;
-
-/**
- * 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
- * {@link InetAddress} objects and finally checkpoints which are identified by
- * <p>
- * This class is thread-safe.
- *
- */
-public class TransferEnvelopeReceiverList {
-
- private final List<ChannelID> localReceivers;
-
- private final List<RemoteReceiver> remoteReceivers;
-
- public TransferEnvelopeReceiverList(final ConnectionInfoLookupResponse cilr) {
-
- this.localReceivers = Collections.unmodifiableList(cilr.getLocalTargets());
- this.remoteReceivers = Collections.unmodifiableList(cilr.getRemoteTargets());
- }
-
- public TransferEnvelopeReceiverList(final ChannelID localReceiver) {
-
- final List<ChannelID> lr = new ArrayList<ChannelID>(1);
- lr.add(localReceiver);
-
- this.localReceivers = Collections.unmodifiableList(lr);
- this.remoteReceivers = Collections.emptyList();
- }
-
- public TransferEnvelopeReceiverList(final RemoteReceiver remoteReceiver) {
-
- final List<RemoteReceiver> rr = new ArrayList<RemoteReceiver>(1);
- rr.add(remoteReceiver);
-
- this.localReceivers = Collections.emptyList();
- this.remoteReceivers = Collections.unmodifiableList(rr);
- }
-
- public boolean hasLocalReceivers() {
-
- return (!this.localReceivers.isEmpty());
- }
-
- public boolean hasRemoteReceivers() {
-
- return (!this.remoteReceivers.isEmpty());
- }
-
- public int getTotalNumberOfReceivers() {
-
- return (this.localReceivers.size() + this.remoteReceivers.size());
- }
-
- public List<RemoteReceiver> getRemoteReceivers() {
-
- return this.remoteReceivers;
- }
-
- public List<ChannelID> getLocalReceivers() {
-
- return this.localReceivers;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/BufferPoolConnector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/BufferPoolConnector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/BufferPoolConnector.java
new file mode 100644
index 0000000..ff1d2be
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/BufferPoolConnector.java
@@ -0,0 +1,45 @@
+/***********************************************************************************************************************
+ * 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.nephele.util;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.BufferRecycler;
+
+import java.util.Queue;
+
+public class BufferPoolConnector implements BufferRecycler {
+
+ /**
+ * Reference to the memory pool the byte buffer was originally taken from.
+ */
+ private final Queue<MemorySegment> memoryPool;
+
+ /**
+ * Constructs a new buffer pool connector
+ *
+ * @param bufferPool
+ * a reference to the memory pool the byte buffer was originally taken from
+ */
+ public BufferPoolConnector(final Queue<MemorySegment> bufferPool) {
+ this.memoryPool = bufferPool;
+ }
+
+ @Override
+ public void recycle(final MemorySegment buffer) {
+ synchronized (this.memoryPool) {
+ this.memoryPool.add(buffer);
+ this.memoryPool.notify();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java
deleted file mode 100644
index 40f6167..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/TaskUtils.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/***********************************************************************************************************************
- * 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.util;
-
-import eu.stratosphere.nephele.annotations.Stateless;
-import eu.stratosphere.nephele.template.AbstractInvokable;
-
-/**
- * This class implements several convenience methods to determine properties of Nephele task classes.
- *
- */
-public class TaskUtils {
-
- /**
- * Private constructor, so class cannot be instantiated.
- */
- private TaskUtils() {
- }
-
- /**
- * Checks if a task is declared to be stateless.
- *
- * @param taskClass
- * the class of the task to check
- * @return <code>true</code> if the given class is declared to be stateless, <code>false</code> otherwise
- */
- public static boolean isStateless(final Class<? extends AbstractInvokable> taskClass) {
-
- return taskClass.isAnnotationPresent(Stateless.class);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/io/FakeOutputTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/io/FakeOutputTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/io/FakeOutputTask.java
index 1795776..ced186b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/io/FakeOutputTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/io/FakeOutputTask.java
@@ -13,7 +13,7 @@
package eu.stratosphere.pact.runtime.iterative.io;
-import eu.stratosphere.nephele.io.MutableRecordReader;
+import eu.stratosphere.runtime.io.api.MutableRecordReader;
import eu.stratosphere.nephele.template.AbstractOutputTask;
import eu.stratosphere.types.Record;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationHeadPactTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationHeadPactTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationHeadPactTask.java
index 2d725b8..c39e3ef 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationHeadPactTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationHeadPactTask.java
@@ -25,7 +25,6 @@ import eu.stratosphere.api.common.typeutils.TypeComparator;
import eu.stratosphere.api.common.typeutils.TypeComparatorFactory;
import eu.stratosphere.api.common.typeutils.TypeSerializer;
import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
-import eu.stratosphere.core.io.IOReadableWritable;
import eu.stratosphere.core.memory.DataInputView;
import eu.stratosphere.core.memory.MemorySegment;
import eu.stratosphere.nephele.io.AbstractRecordWriter;
@@ -78,13 +77,13 @@ public class IterationHeadPactTask<X, Y, S extends Function, OT> extends Abstrac
private Collector<X> finalOutputCollector;
- private List<AbstractRecordWriter<?>> finalOutputWriters;
+ private List<BufferWriter> finalOutputWriters;
private TypeSerializerFactory<Y> feedbackTypeSerializer;
private TypeSerializerFactory<X> solutionTypeSerializer;
- private RecordWriter<?> toSync;
+ private BufferWriter toSync;
private int initialSolutionSetInput; // undefined for bulk iterations
@@ -108,7 +107,7 @@ public class IterationHeadPactTask<X, Y, S extends Function, OT> extends Abstrac
// at this time, the outputs to the step function are created
// add the outputs for the final solution
- this.finalOutputWriters = new ArrayList<AbstractRecordWriter<?>>();
+ this.finalOutputWriters = new ArrayList<BufferWriter>();
final TaskConfig finalOutConfig = this.config.getIterationHeadFinalOutputConfig();
this.finalOutputCollector = RegularPactTask.getOutputCollector(this, finalOutConfig,
this.userCodeClassLoader, this.finalOutputWriters, finalOutConfig.getNumOutputs());
@@ -122,7 +121,7 @@ public class IterationHeadPactTask<X, Y, S extends Function, OT> extends Abstrac
throw new Exception("Error: Inconsistent head task setup - wrong mapping of output gates.");
}
// now, we can instantiate the sync gate
- this.toSync = new RecordWriter<IOReadableWritable>(this, IOReadableWritable.class);
+ this.toSync = new BufferWriter(this);
}
/**
@@ -203,6 +202,8 @@ public class IterationHeadPactTask<X, Y, S extends Function, OT> extends Abstrac
@Override
public void run() throws Exception {
+ // initialize the serializers (one per channel) of the record writers
+ RegularPactTask.initOutputWriters(this.finalOutputWriters);
final String brokerKey = brokerKey();
final int workerIndex = getEnvironment().getIndexInSubtaskGroup();
@@ -330,6 +331,8 @@ public class IterationHeadPactTask<X, Y, S extends Function, OT> extends Abstrac
streamOutFinalOutputBulk(new InputViewIterator<X>(superstepResult, this.solutionTypeSerializer.getSerializer()));
}
+ this.finalOutputCollector.close();
+
} finally {
// make sure we unregister everything from the broker:
// - backchannel
@@ -388,7 +391,8 @@ public class IterationHeadPactTask<X, Y, S extends Function, OT> extends Abstrac
if (log.isInfoEnabled()) {
log.info(formatLogString("sending " + WorkerDoneEvent.class.getSimpleName() + " to sync"));
}
- this.toSync.publishEvent(event);
+
+ this.toSync.broadcastEvent(event);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationIntermediatePactTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationIntermediatePactTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationIntermediatePactTask.java
index 4ada733..5bdb3aa 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationIntermediatePactTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationIntermediatePactTask.java
@@ -19,7 +19,10 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import eu.stratosphere.api.common.functions.Function;
-import eu.stratosphere.nephele.io.AbstractRecordWriter;
+import eu.stratosphere.runtime.io.api.BufferWriter;
+import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannel;
+import eu.stratosphere.runtime.io.channels.EndOfSuperstepEvent;
+import eu.stratosphere.pact.runtime.iterative.event.TerminationEvent;
import eu.stratosphere.pact.runtime.iterative.io.WorksetUpdateOutputCollector;
import eu.stratosphere.util.Collector;
@@ -107,7 +110,7 @@ public class IterationIntermediatePactTask<S extends Function, OT> extends Abstr
}
private void sendEndOfSuperstep() throws IOException, InterruptedException {
- for (AbstractRecordWriter<?> eventualOutput : eventualOutputs) {
+ for (BufferWriter eventualOutput : this.eventualOutputs) {
eventualOutput.sendEndOfSuperstep();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationSynchronizationSinkTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationSynchronizationSinkTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationSynchronizationSinkTask.java
index 35786a7..94f9b9f 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationSynchronizationSinkTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationSynchronizationSinkTask.java
@@ -28,7 +28,7 @@ import eu.stratosphere.api.common.aggregators.AggregatorWithName;
import eu.stratosphere.api.common.aggregators.ConvergenceCriterion;
import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.io.MutableRecordReader;
+import eu.stratosphere.runtime.io.api.MutableRecordReader;
import eu.stratosphere.nephele.template.AbstractOutputTask;
import eu.stratosphere.nephele.types.IntegerRecord;
import eu.stratosphere.pact.runtime.iterative.event.AllWorkersDoneEvent;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationTailPactTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationTailPactTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationTailPactTask.java
index 5cfe173..859a62d 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationTailPactTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationTailPactTask.java
@@ -106,6 +106,7 @@ public class IterationTailPactTask<S extends Function, OT> extends AbstractItera
// aggregate workset update element count
long numCollected = worksetUpdateOutputCollector.getElementsCollectedAndReset();
worksetAggregator.aggregate(numCollected);
+
}
if (log.isInfoEnabled()) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputCollector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputCollector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputCollector.java
index f1c945d..a91a59a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputCollector.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputCollector.java
@@ -19,7 +19,7 @@ import java.util.Collections;
import java.util.List;
import eu.stratosphere.api.common.typeutils.TypeSerializer;
-import eu.stratosphere.nephele.io.AbstractRecordWriter;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.pact.runtime.plugable.SerializationDelegate;
import eu.stratosphere.util.Collector;
@@ -30,7 +30,7 @@ import eu.stratosphere.util.Collector;
public class OutputCollector<T> implements Collector<T>
{
// list of writers
- protected AbstractRecordWriter<SerializationDelegate<T>>[] writers;
+ protected RecordWriter<SerializationDelegate<T>>[] writers;
private final SerializationDelegate<T> delegate;
@@ -43,10 +43,10 @@ public class OutputCollector<T> implements Collector<T>
* @param writers List of all writers.
*/
@SuppressWarnings("unchecked")
- public OutputCollector(List<AbstractRecordWriter<SerializationDelegate<T>>> writers, TypeSerializer<T> serializer)
+ public OutputCollector(List<RecordWriter<SerializationDelegate<T>>> writers, TypeSerializer<T> serializer)
{
this.delegate = new SerializationDelegate<T>(serializer);
- this.writers = (AbstractRecordWriter<SerializationDelegate<T>>[]) writers.toArray(new AbstractRecordWriter[writers.size()]);
+ this.writers = (RecordWriter<SerializationDelegate<T>>[]) writers.toArray(new RecordWriter[writers.size()]);
}
/**
@@ -56,14 +56,14 @@ public class OutputCollector<T> implements Collector<T>
*/
@SuppressWarnings("unchecked")
- public void addWriter(AbstractRecordWriter<SerializationDelegate<T>> writer)
+ public void addWriter(RecordWriter<SerializationDelegate<T>> writer)
{
// avoid using the array-list here to reduce one level of object indirection
if (this.writers == null) {
- this.writers = new AbstractRecordWriter[] {writer};
+ this.writers = new RecordWriter[] {writer};
}
else {
- AbstractRecordWriter<SerializationDelegate<T>>[] ws = new AbstractRecordWriter[this.writers.length + 1];
+ RecordWriter<SerializationDelegate<T>>[] ws = new RecordWriter[this.writers.length + 1];
System.arraycopy(this.writers, 0, ws, 0, this.writers.length);
ws[this.writers.length] = writer;
this.writers = ws;
@@ -79,7 +79,7 @@ public class OutputCollector<T> implements Collector<T>
this.delegate.setInstance(record);
try {
for (int i = 0; i < writers.length; i++) {
- this.writers[i].emit(this.delegate);
+ this.writers[i].emit(this.delegate);
}
}
catch (IOException e) {
@@ -90,19 +90,24 @@ public class OutputCollector<T> implements Collector<T>
}
}
- /*
- * (non-Javadoc)
- * @see eu.stratosphere.pact.common.stub.Collector#close()
- */
@Override
public void close() {
+ for (RecordWriter<?> writer : writers) {
+ try {
+ writer.flush();
+ } catch (IOException e) {
+ throw new RuntimeException(e.getMessage(), e);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e.getMessage(), e);
+ }
+ }
}
/**
* List of writers that are associated with this output collector
* @return list of writers
*/
- public List<AbstractRecordWriter<SerializationDelegate<T>>> getWriters() {
+ public List<RecordWriter<SerializationDelegate<T>>> getWriters() {
return Collections.unmodifiableList(Arrays.asList(this.writers));
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
index ebc7ac8..46e3249 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
@@ -15,7 +15,7 @@ package eu.stratosphere.pact.runtime.shipping;
import eu.stratosphere.api.common.distributions.DataDistribution;
import eu.stratosphere.api.common.typeutils.TypeComparator;
-import eu.stratosphere.nephele.io.ChannelSelector;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
import eu.stratosphere.pact.runtime.plugable.SerializationDelegate;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputCollector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputCollector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputCollector.java
index 852fd3a..4394483 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputCollector.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputCollector.java
@@ -18,7 +18,7 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.List;
-import eu.stratosphere.nephele.io.AbstractRecordWriter;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.types.Record;
import eu.stratosphere.util.Collector;
@@ -27,38 +27,37 @@ import eu.stratosphere.util.Collector;
* The OutputCollector tracks to which writers a deep-copy must be given and which not.
*/
public class RecordOutputCollector implements Collector<Record>
-{
+{
// list of writers
- protected AbstractRecordWriter<Record>[] writers;
-
+ protected RecordWriter<Record>[] writers;
+
/**
- * Initializes the output collector with a set of writers.
- * To specify for a writer that it must be fed with a deep-copy, set the bit in the copy flag bit mask to 1 that
+ * Initializes the output collector with a set of writers.
+ * To specify for a writer that it must be fed with a deep-copy, set the bit in the copy flag bit mask to 1 that
* corresponds to the position of the writer within the {@link List}.
- *
+ *
* @param writers List of all writers.
*/
@SuppressWarnings("unchecked")
- public RecordOutputCollector(List<AbstractRecordWriter<Record>> writers) {
-
- this.writers = (AbstractRecordWriter<Record>[]) writers.toArray(new AbstractRecordWriter[writers.size()]);
+ public RecordOutputCollector(List<RecordWriter<Record>> writers) {
+
+ this.writers = (RecordWriter<Record>[]) writers.toArray(new RecordWriter[writers.size()]);
}
-
+
/**
* Adds a writer to the OutputCollector.
- *
+ *
* @param writer The writer to add.
*/
-
@SuppressWarnings("unchecked")
- public void addWriter(AbstractRecordWriter<Record> writer)
+ public void addWriter(RecordWriter<Record> writer)
{
// avoid using the array-list here to reduce one level of object indirection
if (this.writers == null) {
- this.writers = new AbstractRecordWriter[] {writer};
+ this.writers = new RecordWriter[] {writer};
}
else {
- AbstractRecordWriter<Record>[] ws = new AbstractRecordWriter[this.writers.length + 1];
+ RecordWriter<Record>[] ws = new RecordWriter[this.writers.length + 1];
System.arraycopy(this.writers, 0, ws, 0, this.writers.length);
ws[this.writers.length] = writer;
this.writers = ws;
@@ -74,7 +73,7 @@ public class RecordOutputCollector implements Collector<Record>
{
try {
for (int i = 0; i < writers.length; i++) {
- this.writers[i].emit(record);
+ this.writers[i].emit(record);
}
}
catch (IOException e) {
@@ -85,19 +84,24 @@ public class RecordOutputCollector implements Collector<Record>
}
}
- /*
- * (non-Javadoc)
- * @see eu.stratosphere.pact.common.stub.Collector#close()
- */
@Override
public void close() {
+ for (RecordWriter<?> writer : writers) {
+ try {
+ writer.flush();
+ } catch (IOException e) {
+ throw new RuntimeException(e.getMessage(), e);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e.getMessage(), e);
+ }
+ }
}
/**
* List of writers that are associated with this output collector
* @return list of writers
*/
- public List<AbstractRecordWriter<Record>> getWriters() {
+ public List<RecordWriter<Record>> getWriters() {
return Collections.unmodifiableList(Arrays.asList(writers));
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
index 047c1bf..ba352eb 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
@@ -16,6 +16,8 @@ package eu.stratosphere.pact.runtime.shipping;
import eu.stratosphere.api.common.distributions.DataDistribution;
import eu.stratosphere.api.common.typeutils.TypeComparator;
import eu.stratosphere.nephele.io.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.Record;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
index 250efcc..cb3e782 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
@@ -15,6 +15,7 @@ package eu.stratosphere.pact.runtime.task;
import java.io.IOException;
+import eu.stratosphere.pact.runtime.task.chaining.ExceptionInChainedStubException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -29,10 +30,11 @@ import eu.stratosphere.core.fs.FileSystem;
import eu.stratosphere.core.fs.FileSystem.WriteMode;
import eu.stratosphere.core.fs.Path;
import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.execution.CancelTaskException;
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.io.MutableReader;
-import eu.stratosphere.nephele.io.MutableRecordReader;
-import eu.stratosphere.nephele.io.MutableUnionRecordReader;
+import eu.stratosphere.runtime.io.api.MutableReader;
+import eu.stratosphere.runtime.io.api.MutableRecordReader;
+import eu.stratosphere.runtime.io.api.MutableUnionRecordReader;
import eu.stratosphere.nephele.template.AbstractOutputTask;
import eu.stratosphere.pact.runtime.plugable.DeserializationDelegate;
import eu.stratosphere.pact.runtime.sort.UnilateralSortMerger;
@@ -183,11 +185,16 @@ public class DataSinkTask<IT> extends AbstractOutputTask {
}
}
catch (Exception ex) {
+ ex = ExceptionInChainedStubException.exceptionUnwrap(ex);
+
+ if (ex instanceof CancelTaskException) {
+ // forward canceling exception
+ throw ex;
+ }
// drop, if the task was canceled
- if (!this.taskCanceled) {
- if (LOG.isErrorEnabled()) {
+ else if (!this.taskCanceled) {
+ if (LOG.isErrorEnabled())
LOG.error(getLogString("Error in user code: " + ex.getMessage()), ex);
- }
throw ex;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java
index 47a8218..af176b9 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSourceTask.java
@@ -17,7 +17,10 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
+import java.util.List;
+import eu.stratosphere.pact.runtime.task.chaining.ExceptionInChainedStubException;
+import eu.stratosphere.runtime.io.api.BufferWriter;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -27,6 +30,7 @@ import eu.stratosphere.api.common.typeutils.TypeSerializer;
import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.core.io.InputSplit;
+import eu.stratosphere.nephele.execution.CancelTaskException;
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
import eu.stratosphere.nephele.template.AbstractInputTask;
import eu.stratosphere.pact.runtime.shipping.OutputCollector;
@@ -41,13 +45,15 @@ import eu.stratosphere.util.Collector;
* DataSourceTask which is executed by a Nephele task manager. The task reads data and uses an
* {@link InputFormat} to create records from the input.
*
- * @see eu.stratosphere.api.io.InputFormat
+ * @see eu.stratosphere.api.common.io.InputFormat
*/
public class DataSourceTask<OT> extends AbstractInputTask<InputSplit> {
// Obtain DataSourceTask Logger
private static final Log LOG = LogFactory.getLog(DataSourceTask.class);
+ private List<BufferWriter> eventualOutputs;
+
// Output collector
private Collector<OT> output;
@@ -116,6 +122,9 @@ public class DataSourceTask<OT> extends AbstractInputTask<InputSplit> {
final TypeSerializer<OT> serializer = this.serializerFactory.getSerializer();
try {
+ // initialize the serializers (one per channel) of the record writers
+ RegularPactTask.initOutputWriters(this.eventualOutputs);
+
// start all chained tasks
RegularPactTask.openChainedTasks(this.chainedTasks, this);
@@ -249,8 +258,14 @@ public class DataSourceTask<OT> extends AbstractInputTask<InputSplit> {
RegularPactTask.cancelChainedTasks(this.chainedTasks);
- // drop exception, if the task was canceled
- if (!this.taskCanceled) {
+ ex = ExceptionInChainedStubException.exceptionUnwrap(ex);
+
+ if (ex instanceof CancelTaskException) {
+ // forward canceling exception
+ throw ex;
+ }
+ else if (!this.taskCanceled) {
+ // drop exception, if the task was canceled
RegularPactTask.logAndThrowException(ex, this);
}
}
@@ -315,7 +330,8 @@ public class DataSourceTask<OT> extends AbstractInputTask<InputSplit> {
*/
private void initOutputs(ClassLoader cl) throws Exception {
this.chainedTasks = new ArrayList<ChainedDriver<?, ?>>();
- this.output = RegularPactTask.initOutputs(this, cl, this.config, this.chainedTasks, null);
+ this.eventualOutputs = new ArrayList<BufferWriter>();
+ this.output = RegularPactTask.initOutputs(this, cl, this.config, this.chainedTasks, this.eventualOutputs);
}
// ------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
index 7aed8c0..b01799a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
@@ -1,5 +1,5 @@
/***********************************************************************************************************************
- * 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
* the License. You may obtain a copy of the License at
@@ -13,14 +13,6 @@
package eu.stratosphere.pact.runtime.task;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
import eu.stratosphere.api.common.accumulators.Accumulator;
import eu.stratosphere.api.common.accumulators.AccumulatorHelper;
import eu.stratosphere.api.common.distributions.DataDistribution;
@@ -34,15 +26,15 @@ import eu.stratosphere.configuration.ConfigConstants;
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.configuration.GlobalConfiguration;
import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.execution.CancelTaskException;
import eu.stratosphere.nephele.execution.Environment;
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.io.AbstractRecordWriter;
-import eu.stratosphere.nephele.io.BroadcastRecordWriter;
-import eu.stratosphere.nephele.io.ChannelSelector;
-import eu.stratosphere.nephele.io.MutableReader;
-import eu.stratosphere.nephele.io.MutableRecordReader;
-import eu.stratosphere.nephele.io.MutableUnionRecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
+import eu.stratosphere.runtime.io.api.RecordWriter;
+import eu.stratosphere.runtime.io.api.MutableReader;
+import eu.stratosphere.runtime.io.api.MutableRecordReader;
+import eu.stratosphere.runtime.io.api.MutableUnionRecordReader;
+import eu.stratosphere.runtime.io.api.BufferWriter;
import eu.stratosphere.nephele.services.accumulators.AccumulatorEvent;
import eu.stratosphere.nephele.services.iomanager.IOManager;
import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
@@ -71,18 +63,22 @@ import eu.stratosphere.types.Record;
import eu.stratosphere.util.Collector;
import eu.stratosphere.util.InstantiationUtil;
import eu.stratosphere.util.MutableObjectIterator;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
/**
* The abstract base class for all tasks. Encapsulated common behavior and implements the main life-cycle
* of the user code.
*/
public class RegularPactTask<S extends Function, OT> extends AbstractTask implements PactTaskContext<S, OT> {
-
+
protected static final Log LOG = LogFactory.getLog(RegularPactTask.class);
-
- private static final boolean USE_BROARDCAST_WRITERS = GlobalConfiguration.getBoolean(
- ConfigConstants.USE_MULTICAST_FOR_BROADCAST, ConfigConstants.DEFAULT_USE_MULTICAST_FOR_BROADCAST);
-
+
// --------------------------------------------------------------------------------------------
/**
@@ -95,7 +91,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
* The instantiated user code of this task's main operator (driver). May be null if the operator has no udf.
*/
protected S stub;
-
+
/**
* The udf's runtime context.
*/
@@ -111,13 +107,13 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
* The output writers for the data that this task forwards to the next task. The latest driver (the central, if no chained
* drivers exist, otherwise the last chained driver) produces its output to these writers.
*/
- protected List<AbstractRecordWriter<?>> eventualOutputs;
-
+ protected List<BufferWriter> eventualOutputs;
+
/**
* The input readers to this task.
*/
protected MutableReader<?>[] inputReaders;
-
+
/**
* The input readers for the configured broadcast variables for this task.
*/
@@ -127,7 +123,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
* The inputs reader, wrapped in an iterator. Prior to the local strategies, etc...
*/
protected MutableObjectIterator<?>[] inputIterators;
-
+
/**
* The input readers for the configured broadcast variables, wrapped in an iterator.
* Prior to the local strategies, etc...
@@ -142,18 +138,18 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
* The local strategies that are applied on the inputs.
*/
protected volatile CloseableInputProvider<?>[] localStrategies;
-
+
/**
* The optional temp barriers on the inputs for dead-lock breaking. Are
* optionally resettable.
*/
protected volatile TempBarrier<?>[] tempBarriers;
-
+
/**
* The resettable inputs in the case where no temp barrier is needed.
*/
protected volatile SpillingResettableMutableObjectIterator<?>[] resettableInputs;
-
+
/**
* The inputs to the operator. Return the readers' data after the application of the local strategy
* and the temp-table barrier.
@@ -189,24 +185,24 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
* A list of chained drivers, if there are any.
*/
protected ArrayList<ChainedDriver<?, ?>> chainedTasks;
-
+
/**
* Certain inputs may be excluded from resetting. For example, the initial partial solution
* in an iteration head must not be reseted (it is read through the back channel), when all
* others are reseted.
*/
private boolean[] excludeFromReset;
-
+
/**
* Flag indicating for each input whether it is cached and can be reseted.
*/
private boolean[] inputIsCached;
-
+
/**
* flag indicating for each input whether it must be asynchronously materialized.
*/
private boolean[] inputIsAsyncMaterialized;
-
+
/**
* The amount of memory per input that is dedicated to the materialization.
*/
@@ -283,16 +279,18 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
*/
@Override
public void invoke() throws Exception {
-
- if (LOG.isDebugEnabled()) {
+
+ if (LOG.isDebugEnabled())
LOG.debug(formatLogString("Start task code."));
- }
-
+
// 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,
// so that the thread that creates them catches its own errors that may happen in that process.
// this is especially important, since there may be asynchronous closes (such as through canceling).
try {
+ // initialize the serializers (one per channel) of the record writers
+ initOutputWriters(this.eventualOutputs);
+
// initialize the remaining data structures on the input and trigger the local processing
// the local processing includes building the dams / caches
try {
@@ -352,23 +350,23 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
throw new RuntimeException("Initializing the input processing failed" +
e.getMessage() == null ? "." : ": " + e.getMessage(), e);
}
-
+
if (!this.running) {
if (LOG.isDebugEnabled()) {
LOG.debug(formatLogString("Task cancelled before task code was started."));
}
return;
}
-
+
// pre main-function initialization
initialize();
-
+
// read the broadcast variables
for (int i = 0; i < this.config.getNumBroadcastInputs(); i++) {
final String name = this.config.getBroadcastInputName(i);
readAndSetBroadcastInput(i, name, this.runtimeUdfContext);
}
-
+
// the work goes here
run();
}
@@ -376,7 +374,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
// clean up in any case!
closeLocalStrategiesAndCaches();
}
-
+
if (this.running) {
if (LOG.isDebugEnabled()) {
LOG.debug(formatLogString("Finished task code."));
@@ -387,15 +385,14 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
}
}
}
-
+
@Override
public void cancel() throws Exception {
this.running = false;
-
- if (LOG.isDebugEnabled()) {
+
+ if (LOG.isDebugEnabled())
LOG.debug(formatLogString("Cancelling task code"));
- }
-
+
try {
if (this.driver != null) {
this.driver.cancel();
@@ -434,7 +431,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
// instantiate the UDF
try {
final Class<? super S> userCodeFunctionType = this.driver.getStubType();
- // if the class is null, the driver has no user code
+ // if the class is null, the driver has no user code
if (userCodeFunctionType != null) {
this.stub = initStub(userCodeFunctionType);
}
@@ -462,7 +459,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
}
context.setBroadcastVariable(bcVarName, collection);
}
-
+
protected void run() throws Exception {
// ---------------------------- Now, the actual processing starts ------------------------
// check for asynchronous canceling
@@ -483,7 +480,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
throw new Exception("The data preparation for task '" + this.getEnvironment().getTaskName() +
"' , caused an error: " + t.getMessage(), t);
}
-
+
// check for canceling
if (!this.running) {
return;
@@ -517,7 +514,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
// close all chained tasks letting them report failure
RegularPactTask.closeChainedTasks(this.chainedTasks, this);
-
+
// Collect the accumulators of all involved UDFs and send them to the
// JobManager. close() has been called earlier for all involved UDFs
// (using this.stub.close() and closeChainedTasks()), so UDFs can no longer
@@ -539,8 +536,14 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
RegularPactTask.cancelChainedTasks(this.chainedTasks);
- // drop exception, if the task was canceled
- if (this.running) {
+ ex = ExceptionInChainedStubException.exceptionUnwrap(ex);
+
+ if (ex instanceof CancelTaskException) {
+ // forward canceling exception
+ throw ex;
+ }
+ else if (this.running) {
+ // throw only if task was not cancelled. in the case of canceling, exceptions are expected
RegularPactTask.logAndThrowException(ex, this);
}
}
@@ -553,9 +556,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
* This method is called at the end of a task, receiving the accumulators of
* the task and the chained tasks. It merges them into a single map of
* accumulators and sends them to the JobManager.
- *
- * @param env
- * @param accumulators
+ *
* @param chainedTasks
* Each chained task might have accumulators which will be merged
* with the accumulators of the stub.
@@ -566,7 +567,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
// We can merge here the accumulators from the stub and the chained
// tasks. Type conflicts can occur here if counters with same name but
// different type were used.
-
+
for (ChainedDriver<?, ?> chainedTask : chainedTasks) {
Map<String, Accumulator<?, ?>> chainedAccumulators = chainedTask.getStub().getRuntimeContext().getAllAccumulators();
AccumulatorHelper.mergeInto(accumulators, chainedAccumulators);
@@ -632,8 +633,6 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
}
}
-
-
// --------------------------------------------------------------------------------------------
// Task Setup and Teardown
// --------------------------------------------------------------------------------------------
@@ -665,12 +664,12 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
chainedTasks.get(numChained - 1).setOutputCollector(newOutputCollector);
}
-
+
public TaskConfig getLastTasksConfig() {
int numChained = this.chainedTasks.size();
return (numChained == 0) ? config : chainedTasks.get(numChained - 1).getTaskConfig();
}
-
+
protected S initStub(Class<? super S> stubSuperClass) throws Exception {
try {
S stub = config.<S>getStubWrapper(this.userCodeClassLoader).getUserCodeObject(stubSuperClass, this.userCodeClassLoader);
@@ -687,7 +686,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
throw new Exception("The stub class is not a proper subclass of " + stubSuperClass.getName(), ccex);
}
}
-
+
/**
* Creates the record readers for the number of inputs as defined by {@link #getNumTaskInputs()}.
*
@@ -697,9 +696,9 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
protected void initInputReaders() throws Exception {
final int numInputs = getNumTaskInputs();
final MutableReader<?>[] inputReaders = new MutableReader[numInputs];
-
+
int numGates = 0;
-
+
for (int i = 0; i < numInputs; i++) {
// ---------------- create the input readers ---------------------
// in case where a logical input unions multiple physical inputs, create a union reader
@@ -720,13 +719,13 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
}
}
this.inputReaders = inputReaders;
-
+
// final sanity check
if (numGates != this.config.getNumInputs()) {
throw new Exception("Illegal configuration: Number of input gates and group sizes are not consistent.");
}
}
-
+
/**
* Creates the record readers for the extra broadcast inputs as configured by {@link TaskConfig#getNumBroadcastInputs()}.
*
@@ -776,7 +775,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
final TypeComparatorFactory<?> comparatorFactory = this.config.getDriverComparator(i, this.userCodeClassLoader);
this.inputComparators[i] = comparatorFactory.createComparator();
}
-
+
this.inputIterators[i] = createInputIterator(this.inputReaders[i], this.inputSerializers[i]);
}
}
@@ -796,34 +795,34 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
this.broadcastInputIterators[i] = createInputIterator(this.broadcastInputReaders[i], this.broadcastInputSerializers[i]);
}
}
-
+
/**
- *
+ *
* NOTE: This method must be invoked after the invocation of {@code #initInputReaders()} and
* {@code #initInputSerializersAndComparators(int)}!
- *
+ *
* @param numInputs
*/
protected void initLocalStrategies(int numInputs) throws Exception {
-
+
final MemoryManager memMan = getMemoryManager();
final IOManager ioMan = getIOManager();
-
+
this.localStrategies = new CloseableInputProvider[numInputs];
this.inputs = new MutableObjectIterator[numInputs];
this.excludeFromReset = new boolean[numInputs];
this.inputIsCached = new boolean[numInputs];
this.inputIsAsyncMaterialized = new boolean[numInputs];
this.materializationMemory = new int[numInputs];
-
+
// set up the local strategies first, such that the can work before any temp barrier is created
for (int i = 0; i < numInputs; i++) {
initInputLocalStrategy(i);
}
-
+
// we do another loop over the inputs, because we want to instantiate all
// sorters, etc before requesting the first input (as this call may block)
-
+
// we have two types of materialized inputs, and both are replayable (can act as a cache)
// The first variant materializes in a different thread and hence
// acts as a pipeline breaker. this one should only be there, if a pipeline breaker is needed.
@@ -831,15 +830,15 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
// in a pipelined fashion.
this.resettableInputs = new SpillingResettableMutableObjectIterator[numInputs];
this.tempBarriers = new TempBarrier[numInputs];
-
+
for (int i = 0; i < numInputs; i++) {
final int memoryPages;
final boolean async = this.config.isInputAsynchronouslyMaterialized(i);
final boolean cached = this.config.isInputCached(i);
-
+
this.inputIsAsyncMaterialized[i] = async;
this.inputIsCached[i] = cached;
-
+
if (async || cached) {
memoryPages = memMan.computeNumberOfPages(this.config.getInputMaterializationMemory(i));
if (memoryPages <= 0) {
@@ -849,7 +848,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
} else {
memoryPages = 0;
}
-
+
if (async) {
@SuppressWarnings({ "unchecked", "rawtypes" })
TempBarrier<?> barrier = new TempBarrier(this, getInput(i), this.inputSerializers[i], memMan, ioMan, memoryPages);
@@ -865,7 +864,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
}
}
}
-
+
protected void resetAllInputs() throws Exception {
// close all local-strategies. they will either get re-initialized, or we have
// read them now and their data is cached
@@ -875,10 +874,10 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
this.localStrategies[i] = null;
}
}
-
+
final MemoryManager memMan = getMemoryManager();
final IOManager ioMan = getIOManager();
-
+
// reset the caches, or re-run the input local strategy
for (int i = 0; i < this.inputs.length; i++) {
if (this.excludeFromReset[i]) {
@@ -892,7 +891,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
} else {
// make sure the input is not available directly, but are lazily fetched again
this.inputs[i] = null;
-
+
if (this.inputIsCached[i]) {
if (this.tempBarriers[i] != null) {
this.inputs[i] = this.tempBarriers[i].getIterator();
@@ -908,10 +907,10 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
if (this.tempBarriers[i] != null) {
this.tempBarriers[i].close();
}
-
+
// recreate the local strategy
initInputLocalStrategy(i);
-
+
if (this.inputIsAsyncMaterialized[i]) {
final int pages = this.materializationMemory[i];
@SuppressWarnings({ "unchecked", "rawtypes" })
@@ -924,17 +923,17 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
}
}
}
-
+
protected void excludeFromReset(int inputNum) {
this.excludeFromReset[inputNum] = true;
}
-
+
private void initInputLocalStrategy(int inputNum) throws Exception {
// check if there is already a strategy
if (this.localStrategies[inputNum] != null) {
throw new IllegalStateException();
}
-
+
// now set up the local strategy
final LocalStrategy localStrategy = this.config.getInputLocalStrategy(inputNum);
if (localStrategy != null) {
@@ -960,7 +959,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
if (inputNum != 0) {
throw new IllegalStateException("Performing combining sort outside a (group)reduce task!");
}
-
+
// instantiate ourselves a combiner. we should not use the stub, because the sort and the
// subsequent (group)reduce would otherwise share it multi-threaded
final Class<S> userCodeFunctionType = this.driver.getStubType();
@@ -986,7 +985,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
this.config.getMemoryInput(inputNum), this.config.getFilehandlesInput(inputNum),
this.config.getSpillingThresholdInput(inputNum));
cSorter.setUdfConfiguration(this.config.getStubParameters());
-
+
// set the input to null such that it will be lazily fetched from the input strategy
this.inputs[inputNum] = null;
this.localStrategies[inputNum] = cSorter;
@@ -999,7 +998,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
this.inputs[inputNum] = this.inputIterators[inputNum];
}
}
-
+
private <T> TypeComparator<T> getLocalStrategyComparator(int inputNum) throws Exception {
TypeComparatorFactory<T> compFact = this.config.getInputComparator(inputNum, this.userCodeClassLoader);
if (compFact == null) {
@@ -1030,21 +1029,21 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
// final MutableObjectIterator<?> iter = new ReaderIterator(reader, serializer);
// return iter;
}
-
+
protected int getNumTaskInputs() {
return this.driver.getNumberOfInputs();
}
-
+
/**
* Creates a writer for each output. Creates an OutputCollector which forwards its input to all writers.
* The output collector applies the configured shipping strategies for each writer.
*/
protected void initOutputs() throws Exception {
this.chainedTasks = new ArrayList<ChainedDriver<?, ?>>();
- this.eventualOutputs = new ArrayList<AbstractRecordWriter<?>>();
+ this.eventualOutputs = new ArrayList<BufferWriter>();
this.output = initOutputs(this, this.userCodeClassLoader, this.config, this.chainedTasks, this.eventualOutputs);
}
-
+
public RuntimeUDFContext createRuntimeContext(String taskName) {
Environment env = getEnvironment();
return new RuntimeUDFContext(taskName, env.getCurrentNumberOfSubtasks(), env.getIndexInSubtaskGroup(), env.getCopyTask());
@@ -1054,61 +1053,52 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
// Task Context Signature
// -------------------------------------------------------------------------------------------
-
@Override
public TaskConfig getTaskConfig() {
return this.config;
}
-
@Override
public ClassLoader getUserCodeClassLoader() {
return this.userCodeClassLoader;
}
-
@Override
public MemoryManager getMemoryManager() {
return getEnvironment().getMemoryManager();
}
-
@Override
public IOManager getIOManager() {
return getEnvironment().getIOManager();
}
-
@Override
public S getStub() {
return this.stub;
}
-
@Override
public Collector<OT> getOutputCollector() {
return this.output;
}
-
@Override
public AbstractInvokable getOwningNepheleTask() {
return this;
}
-
@Override
public String formatLogString(String message) {
return constructLogString(message, getEnvironment().getTaskName(), this);
}
-
@Override
public <X> MutableObjectIterator<X> getInput(int index) {
if (index < 0 || index > this.driver.getNumberOfInputs()) {
throw new IndexOutOfBoundsException();
}
-
+
// check for lazy assignment from input strategies
if (this.inputs[index] != null) {
@SuppressWarnings("unchecked")
@@ -1235,19 +1225,19 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
*
* @return The OutputCollector that data produced in this task is submitted to.
*/
- public static <T> Collector<T> getOutputCollector(AbstractInvokable task, TaskConfig config, ClassLoader cl, List<AbstractRecordWriter<?>> eventualOutputs, int numOutputs)
+ public static <T> Collector<T> getOutputCollector(AbstractInvokable task, TaskConfig config, ClassLoader cl, List<BufferWriter> eventualOutputs, int numOutputs)
throws Exception
{
if (numOutputs <= 0) {
throw new Exception("BUG: The task must have at least one output");
}
-
+
// get the factory for the serializer
final TypeSerializerFactory<T> serializerFactory = config.getOutputSerializer(cl);
// special case the Record
if (serializerFactory.getDataType().equals(Record.class)) {
- final List<AbstractRecordWriter<Record>> writers = new ArrayList<AbstractRecordWriter<Record>>(numOutputs);
+ final List<RecordWriter<Record>> writers = new ArrayList<RecordWriter<Record>>(numOutputs);
// create a writer for each output
for (int i = 0; i < numOutputs; i++) {
@@ -1267,18 +1257,10 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
oe = new RecordOutputEmitter(strategy, comparator, distribution);
}
- if (strategy == ShipStrategyType.BROADCAST && USE_BROARDCAST_WRITERS) {
- if (task instanceof AbstractTask) {
- writers.add(new BroadcastRecordWriter<Record>((AbstractTask) task, Record.class));
- } else if (task instanceof AbstractInputTask<?>) {
- writers.add(new BroadcastRecordWriter<Record>((AbstractInputTask<?>) task, Record.class));
- }
- } else {
- if (task instanceof AbstractTask) {
- writers.add(new RecordWriter<Record>((AbstractTask) task, Record.class, oe));
- } else if (task instanceof AbstractInputTask<?>) {
- writers.add(new RecordWriter<Record>((AbstractInputTask<?>) task, Record.class, oe));
- }
+ if (task instanceof AbstractTask) {
+ writers.add(new RecordWriter<Record>((AbstractTask) task, oe));
+ } else if (task instanceof AbstractInputTask<?>) {
+ writers.add(new RecordWriter<Record>((AbstractInputTask<?>) task, oe));
}
}
if (eventualOutputs != null) {
@@ -1291,9 +1273,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
}
else {
// generic case
- final List<AbstractRecordWriter<SerializationDelegate<T>>> writers = new ArrayList<AbstractRecordWriter<SerializationDelegate<T>>>(numOutputs);
- @SuppressWarnings("unchecked") // uncritical, simply due to broken generics
- final Class<SerializationDelegate<T>> delegateClazz = (Class<SerializationDelegate<T>>) (Class<?>) SerializationDelegate.class;
+ final List<RecordWriter<SerializationDelegate<T>>> writers = new ArrayList<RecordWriter<SerializationDelegate<T>>>(numOutputs);
// create a writer for each output
for (int i = 0; i < numOutputs; i++)
@@ -1314,18 +1294,10 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
oe = new OutputEmitter<T>(strategy, comparator, dataDist);
}
- if (strategy == ShipStrategyType.BROADCAST && USE_BROARDCAST_WRITERS) {
- if (task instanceof AbstractTask) {
- writers.add(new BroadcastRecordWriter<SerializationDelegate<T>>((AbstractTask) task, delegateClazz));
- } else if (task instanceof AbstractInputTask<?>) {
- writers.add(new BroadcastRecordWriter<SerializationDelegate<T>>((AbstractInputTask<?>) task, delegateClazz));
- }
- } else {
- if (task instanceof AbstractTask) {
- writers.add(new RecordWriter<SerializationDelegate<T>>((AbstractTask) task, delegateClazz, oe));
- } else if (task instanceof AbstractInputTask<?>) {
- writers.add(new RecordWriter<SerializationDelegate<T>>((AbstractInputTask<?>) task, delegateClazz, oe));
- }
+ if (task instanceof AbstractTask) {
+ writers.add(new RecordWriter<SerializationDelegate<T>>((AbstractTask) task, oe));
+ } else if (task instanceof AbstractInputTask<?>) {
+ writers.add(new RecordWriter<SerializationDelegate<T>>((AbstractInputTask<?>) task, oe));
}
}
if (eventualOutputs != null) {
@@ -1341,7 +1313,7 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
*/
@SuppressWarnings("unchecked")
public static <T> Collector<T> initOutputs(AbstractInvokable nepheleTask, ClassLoader cl, TaskConfig config,
- List<ChainedDriver<?, ?>> chainedTasksTarget, List<AbstractRecordWriter<?>> eventualOutputs)
+ List<ChainedDriver<?, ?>> chainedTasksTarget, List<BufferWriter> eventualOutputs)
throws Exception
{
final int numOutputs = config.getNumOutputs();
@@ -1391,6 +1363,12 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
// instantiate the output collector the default way from this configuration
return getOutputCollector(nepheleTask , config, cl, eventualOutputs, numOutputs);
}
+
+ public static void initOutputWriters(List<BufferWriter> writers) {
+ for (BufferWriter writer : writers) {
+ ((RecordWriter<?>) writer).initializeSerializers();
+ }
+ }
// --------------------------------------------------------------------------------------------
// User Code LifeCycle
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ExceptionInChainedStubException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ExceptionInChainedStubException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ExceptionInChainedStubException.java
index 568bb7d..786928b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ExceptionInChainedStubException.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/ExceptionInChainedStubException.java
@@ -47,4 +47,12 @@ public class ExceptionInChainedStubException extends RuntimeException
public Exception getWrappedException() {
return exception;
}
+
+ public static Exception exceptionUnwrap(Exception e) {
+ if (e instanceof ExceptionInChainedStubException) {
+ return exceptionUnwrap(((ExceptionInChainedStubException) e).getWrappedException());
+ }
+
+ return e;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/SynchronousChainedCombineDriver.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/SynchronousChainedCombineDriver.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/SynchronousChainedCombineDriver.java
index c5c26eb..814eb62 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/SynchronousChainedCombineDriver.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/chaining/SynchronousChainedCombineDriver.java
@@ -165,6 +165,8 @@ public class SynchronousChainedCombineDriver<T> extends ChainedDriver<T, T> {
} catch (Exception e) {
throw new ExceptionInChainedStubException(this.taskName, e);
}
+
+ this.outputCollector.close();
}
private void sortAndCombine() throws Exception {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/ReaderIterator.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/ReaderIterator.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/ReaderIterator.java
index c25a106..b02850c 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/ReaderIterator.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/ReaderIterator.java
@@ -16,7 +16,7 @@ package eu.stratosphere.pact.runtime.task.util;
import java.io.IOException;
import eu.stratosphere.api.common.typeutils.TypeSerializer;
-import eu.stratosphere.nephele.io.MutableReader;
+import eu.stratosphere.runtime.io.api.MutableReader;
import eu.stratosphere.pact.runtime.plugable.DeserializationDelegate;
import eu.stratosphere.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/RecordReaderIterator.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/RecordReaderIterator.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/RecordReaderIterator.java
index 7e3ce5f..b087c2c 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/RecordReaderIterator.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/RecordReaderIterator.java
@@ -15,7 +15,7 @@ package eu.stratosphere.pact.runtime.task.util;
import java.io.IOException;
-import eu.stratosphere.nephele.io.MutableReader;
+import eu.stratosphere.runtime.io.api.MutableReader;
import eu.stratosphere.types.Record;
import eu.stratosphere.util.MutableObjectIterator;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java
new file mode 100644
index 0000000..c192cb9
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java
@@ -0,0 +1,93 @@
+/***********************************************************************************************************************
+ * 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;
+
+import eu.stratosphere.core.memory.MemorySegment;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class Buffer {
+
+ private final MemorySegment memorySegment;
+
+ private final BufferRecycler recycler;
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ private final AtomicInteger referenceCounter;
+
+ private int size;
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ public Buffer(MemorySegment memorySegment, int size, BufferRecycler recycler) {
+ this.memorySegment = memorySegment;
+ this.size = size;
+ this.recycler = recycler;
+
+ // we are the first, so we start with reference count of one
+ this.referenceCounter = new AtomicInteger(1);
+ }
+
+ /**
+ * NOTE: Requires that the reference counter was increased prior to the constructor call!
+ *
+ * @param toDuplicate Buffer instance to duplicate
+ */
+ private Buffer(Buffer toDuplicate) {
+ if (toDuplicate.referenceCounter.getAndIncrement() == 0) {
+ throw new IllegalStateException("Buffer was released before duplication.");
+ }
+
+ this.memorySegment = toDuplicate.memorySegment;
+ this.size = toDuplicate.size;
+ this.recycler = toDuplicate.recycler;
+ this.referenceCounter = toDuplicate.referenceCounter;
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ public MemorySegment getMemorySegment() {
+ return this.memorySegment;
+ }
+
+ public int size() {
+ return this.size;
+ }
+
+ public void limitSize(int size) {
+ if (size >= 0 && size <= this.memorySegment.size()) {
+ this.size = size;
+ } else {
+ throw new IllegalArgumentException();
+ }
+ }
+
+ public void recycleBuffer() {
+ if (this.referenceCounter.decrementAndGet() == 0) {
+ this.recycler.recycle(this.memorySegment);
+ }
+ }
+
+ public Buffer duplicate() {
+ return new Buffer(this);
+ }
+
+ public void copyToBuffer(Buffer destinationBuffer) {
+ if (size() > destinationBuffer.size()) {
+ throw new IllegalArgumentException("Destination buffer is too small to store content of source buffer.");
+ }
+
+ this.memorySegment.copyTo(0, destinationBuffer.memorySegment, 0, size);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/BufferRecycler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/BufferRecycler.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/BufferRecycler.java
new file mode 100644
index 0000000..88f9edb
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/BufferRecycler.java
@@ -0,0 +1,26 @@
+/***********************************************************************************************************************
+ * 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;
+
+import eu.stratosphere.core.memory.MemorySegment;
+
+public interface BufferRecycler {
+
+ /**
+ * Called by {@link eu.stratosphere.runtime.io.Buffer} to return a {@link MemorySegment} to its original buffer pool.
+ *
+ * @param buffer the segment to be recycled
+ */
+ void recycle(MemorySegment buffer);
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractRecordReader.java
new file mode 100644
index 0000000..d71695c
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractRecordReader.java
@@ -0,0 +1,98 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.nephele.event.task.EventListener;
+import eu.stratosphere.nephele.event.task.EventNotificationManager;
+
+/**
+ * This is an abstract base class for a record reader, either dealing with mutable or immutable records,
+ * and dealing with reads from single gates (single end points) or multiple gates (union).
+ */
+public abstract class AbstractRecordReader implements ReaderBase {
+
+
+ private final EventNotificationManager eventHandler = new EventNotificationManager();
+
+ private int numEventsUntilEndOfSuperstep = -1;
+
+ private int endOfSuperstepEventsCount;
+
+ // --------------------------------------------------------------------------------------------
+
+ /**
+ * Subscribes the listener object to receive events of the given type.
+ *
+ * @param eventListener
+ * the listener object to register
+ * @param eventType
+ * the type of event to register the listener for
+ */
+ @Override
+ public void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
+ this.eventHandler.subscribeToEvent(eventListener, eventType);
+ }
+
+ /**
+ * Removes the subscription for events of the given type for the listener object.
+ *
+ * @param eventListener The listener object to cancel the subscription for.
+ * @param eventType The type of the event to cancel the subscription for.
+ */
+ @Override
+ public void unsubscribeFromEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
+ this.eventHandler.unsubscribeFromEvent(eventListener, eventType);
+ }
+
+
+ protected void handleEvent(AbstractTaskEvent evt) {
+ this.eventHandler.deliverEvent(evt);
+ }
+
+ @Override
+ public void setIterative(int numEventsUntilEndOfSuperstep) {
+ this.numEventsUntilEndOfSuperstep = numEventsUntilEndOfSuperstep;
+ }
+
+ @Override
+ public void startNextSuperstep() {
+ if (this.numEventsUntilEndOfSuperstep == -1) {
+ throw new IllegalStateException("Called 'startNextSuperstep()' in a non-iterative reader.");
+ }
+ else if (endOfSuperstepEventsCount < numEventsUntilEndOfSuperstep) {
+ throw new IllegalStateException("Premature 'startNextSuperstep()'. Not yet reached the end-of-superstep.");
+ }
+ this.endOfSuperstepEventsCount = 0;
+ }
+
+ @Override
+ public boolean hasReachedEndOfSuperstep() {
+ return endOfSuperstepEventsCount== numEventsUntilEndOfSuperstep;
+ }
+
+ protected boolean incrementEndOfSuperstepEventAndCheck() {
+ if (numEventsUntilEndOfSuperstep == -1) {
+ throw new IllegalStateException("Received EndOfSuperstep event in a non-iterative reader.");
+ }
+
+ endOfSuperstepEventsCount++;
+
+ if (endOfSuperstepEventsCount > numEventsUntilEndOfSuperstep) {
+ throw new IllegalStateException("Received EndOfSuperstep events beyond the number to indicate the end of the superstep");
+ }
+
+ return endOfSuperstepEventsCount== numEventsUntilEndOfSuperstep;
+ }
+}
[07/30] git commit: Replace custom Java NIO TCP/IP code with Netty 4
library
Posted by rm...@apache.org.
Replace custom Java NIO TCP/IP code with Netty 4 library
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/4cd4a134
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/4cd4a134
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/4cd4a134
Branch: refs/heads/master
Commit: 4cd4a13415d609a2979c8fa3cf4b797c990ee8c2
Parents: 2db78a8
Author: uce <u....@fu-berlin.de>
Authored: Fri May 9 13:39:15 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Sat Jun 7 09:41:21 2014 +0200
----------------------------------------------------------------------
.../configuration/ConfigConstants.java | 58 +-
stratosphere-runtime/pom.xml | 8 +-
.../eu/stratosphere/nephele/AbstractID.java | 6 +
.../nephele/execution/CancelTaskException.java | 3 -
.../nephele/execution/Environment.java | 76 +-
.../execution/ExecutionStateTransition.java | 2 +
.../nephele/execution/RuntimeEnvironment.java | 170 ++--
.../nephele/executiongraph/ExecutionVertex.java | 36 +
.../nephele/instance/AbstractInstance.java | 16 +
.../eu/stratosphere/nephele/jobgraph/JobID.java | 11 +-
.../nephele/jobmanager/JobManager.java | 34 +
.../jobmanager/scheduler/RecoveryLogic.java | 5 +-
.../protocols/ExtendedManagementProtocol.java | 13 +
.../protocols/TaskOperationProtocol.java | 12 +
.../stratosphere/nephele/taskmanager/Task.java | 93 +-
.../nephele/taskmanager/TaskKillResult.java | 44 +
.../nephele/taskmanager/TaskManager.java | 32 +-
.../runtime/ExecutorThreadFactory.java | 35 +
.../task/AbstractIterativePactTask.java | 9 +-
.../iterative/task/IterationHeadPactTask.java | 3 +-
.../pact/runtime/shipping/OutputEmitter.java | 1 -
.../runtime/shipping/RecordOutputEmitter.java | 3 -
.../pact/runtime/task/DataSinkTask.java | 3 +-
.../pact/runtime/task/RegularPactTask.java | 8 +-
.../java/eu/stratosphere/runtime/io/Buffer.java | 5 +-
.../runtime/io/channels/Channel.java | 4 +-
.../runtime/io/channels/ChannelID.java | 10 +-
.../runtime/io/channels/ChannelType.java | 5 +
.../runtime/io/channels/InputChannel.java | 17 +-
.../runtime/io/channels/OutputChannel.java | 5 +-
.../runtime/io/gates/InputGate.java | 7 +-
.../runtime/io/gates/OutputGate.java | 1 -
.../runtime/io/network/ChannelManager.java | 50 +-
.../runtime/io/network/Envelope.java | 178 ++++
.../runtime/io/network/EnvelopeDispatcher.java | 46 +
.../io/network/EnvelopeReceiverList.java | 75 ++
.../io/network/NetworkConnectionManager.java | 176 ----
.../runtime/io/network/SenderHintEvent.java | 1 -
.../BufferAvailabilityListener.java | 17 +-
.../network/bufferprovider/BufferProvider.java | 18 +-
.../bufferprovider/DiscardBufferPool.java | 51 ++
.../network/bufferprovider/LocalBufferPool.java | 26 +-
.../bufferprovider/SerialSingleBufferPool.java | 77 --
.../runtime/io/network/envelope/Envelope.java | 169 ----
.../io/network/envelope/EnvelopeDispatcher.java | 46 -
.../io/network/envelope/EnvelopeReader.java | 212 -----
.../network/envelope/EnvelopeReceiverList.java | 75 --
.../io/network/envelope/EnvelopeWriter.java | 134 ---
.../envelope/NoBufferAvailableException.java | 53 --
.../network/netty/InboundEnvelopeDecoder.java | 344 ++++++++
.../netty/InboundEnvelopeDispatcherHandler.java | 41 +
.../network/netty/NettyConnectionManager.java | 251 ++++++
.../network/netty/OutboundConnectionQueue.java | 94 ++
.../network/netty/OutboundEnvelopeEncoder.java | 65 ++
.../io/network/tcp/IncomingConnection.java | 115 ---
.../network/tcp/IncomingConnectionThread.java | 226 -----
.../io/network/tcp/OutgoingConnection.java | 529 ------------
.../network/tcp/OutgoingConnectionThread.java | 276 ------
.../AdaptiveSpanningRecordDeserializer.java | 32 +-
.../io/serialization/DataInputDeserializer.java | 25 +-
.../io/serialization/DataOutputSerializer.java | 7 +-
.../serialization/SpanningRecordSerializer.java | 7 +-
.../nephele/util/TestBufferProvider.java | 2 +-
.../runtime/task/util/OutputEmitterTest.java | 1 -
.../task/util/RecordOutputEmitterTest.java | 2 -
.../pact/runtime/test/util/MockEnvironment.java | 11 +-
.../envelope/EnvelopeReaderWriterTest.java | 394 ---------
.../netty/InboundEnvelopeDecoderTest.java | 857 +++++++++++++++++++
.../netty/NettyConnectionManagerTest.java | 196 +++++
.../netty/OutboundEnvelopeEncoderTest.java | 97 +++
.../KMeansIterativeNepheleITCase.java | 3 +-
.../clients/examples/LocalExecutorITCase.java | 3 +-
.../exampleJavaPrograms/WordCountITCase.java | 5 +-
.../ConnectedComponentsNepheleITCase.java | 32 +-
.../IterationWithChainingNepheleITCase.java | 12 +-
.../WordCountUnionReduceITCase.java | 4 -
.../test/runtime/NetworkStackNepheleITCase.java | 286 +++++++
77 files changed, 3236 insertions(+), 2820 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java b/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
index 3b9ba3d..96c8965 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
@@ -34,7 +34,6 @@ public final class ConfigConstants {
* The config parameter defining the maximal intra-node parallelism for jobs.
*/
public static final String PARALLELIZATION_MAX_INTRA_NODE_DEGREE_KEY = "parallelization.intra-node.default";
-
// -------------------------------- Runtime -------------------------------
@@ -98,6 +97,26 @@ public final class ConfigConstants {
* The config parameter defining the size of the buffers used in the network stack.
*/
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
@@ -134,10 +153,9 @@ public final class ConfigConstants {
* The parameter defining the polling interval (in seconds) for the JobClient.
*/
public static final String JOBCLIENT_POLLING_INTERVAL_KEY = "jobclient.polling.interval";
-
-
+
// ------------------------ Hadoop Configuration ------------------------
-
+
/**
* Path to hdfs-defaul.xml file
*/
@@ -153,7 +171,6 @@ public final class ConfigConstants {
*/
public static final String PATH_HADOOP_CONFIG = "fs.hdfs.hadoopconf";
-
// ------------------------ File System Bahavior ------------------------
/**
@@ -251,11 +268,7 @@ public final class ConfigConstants {
public static final String STRATOSPHERE_BASE_DIR_PATH_KEY = "stratosphere.base.dir.path";
public static final String STRATOSPHERE_JVM_OPTIONS = "env.java.opts";
-
-
-
-
// ------------------------------------------------------------------------
// Default Values
// ------------------------------------------------------------------------
@@ -318,7 +331,31 @@ public final class ConfigConstants {
* Default size of network stack buffers.
*/
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).
*/
@@ -452,7 +489,6 @@ public final class ConfigConstants {
*/
public static final int DEFAULT_DEFAULT_INSTANCE_TYPE_INDEX = 1;
-
// ------------------------------------------------------------------------
/**
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/pom.xml b/stratosphere-runtime/pom.xml
index bb5ca2a..074725b 100644
--- a/stratosphere-runtime/pom.xml
+++ b/stratosphere-runtime/pom.xml
@@ -52,7 +52,13 @@
<artifactId>aws-java-sdk</artifactId>
<version>1.2.1</version>
</dependency>
-
+
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-all</artifactId>
+ <version>4.0.19.Final</version>
+ </dependency>
+
<dependency>
<groupId>eu.stratosphere</groupId>
<artifactId>stratosphere-java</artifactId>
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java
index 476e22a..648c8dc 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java
@@ -20,6 +20,7 @@ import java.nio.ByteBuffer;
import eu.stratosphere.core.io.IOReadableWritable;
import eu.stratosphere.util.StringUtils;
+import io.netty.buffer.ByteBuf;
/**
* A statistically unique identification number.
@@ -167,6 +168,11 @@ public class AbstractID implements IOReadableWritable {
buffer.putLong(this.upperPart);
}
+ public void writeTo(ByteBuf buf) {
+ buf.writeLong(this.lowerPart);
+ buf.writeLong(this.upperPart);
+ }
+
@Override
public String toString() {
final byte[] ba = new byte[SIZE];
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java
index 6a09e89..aa196da 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/CancelTaskException.java
@@ -14,9 +14,6 @@
**********************************************************************************************************************/
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.
*/
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
index e0bcc70..05a420c 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/Environment.java
@@ -16,17 +16,21 @@ package eu.stratosphere.nephele.execution;
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.core.fs.Path;
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.JobID;
import eu.stratosphere.nephele.protocols.AccumulatorProtocol;
import eu.stratosphere.nephele.services.iomanager.IOManager;
import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
-import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
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
@@ -38,114 +42,112 @@ public interface Environment {
* Returns the ID of the job from the original job graph. It is used by the library cache manager to find the
* required
* libraries for executing the assigned Nephele task.
- *
+ *
* @return the ID of the job from the original job graph
*/
JobID getJobID();
/**
* Returns the task configuration object which was attached to the original JobVertex.
- *
+ *
* @return the task configuration object which was attached to the original JobVertex.
*/
Configuration getTaskConfiguration();
/**
* Returns the job configuration object which was attached to the original {@link JobGraph}.
- *
+ *
* @return the job configuration object which was attached to the original {@link JobGraph}
*/
Configuration getJobConfiguration();
/**
* Returns the current number of subtasks the respective task is split into.
- *
+ *
* @return the current number of subtasks the respective task is split into
*/
int getCurrentNumberOfSubtasks();
/**
* Returns the index of this subtask in the subtask group.
- *
+ *
* @return the index of this subtask in the subtask group
*/
int getIndexInSubtaskGroup();
/**
* Sends a notification that objects that a new user thread has been started to the execution observer.
- *
- * @param userThread
- * the user thread which has been started
+ *
+ * @param userThread the user thread which has been started
*/
void userThreadStarted(Thread userThread);
/**
* Sends a notification that a user thread has finished to the execution observer.
- *
- * @param userThread
- * the user thread which has finished
+ *
+ * @param userThread the user thread which has finished
*/
void userThreadFinished(Thread userThread);
/**
* Returns the input split provider assigned to this environment.
- *
+ *
* @return the input split provider or <code>null</code> if no such provider has been assigned to this environment.
*/
InputSplitProvider getInputSplitProvider();
/**
* Returns the current {@link IOManager}.
- *
+ *
* @return the current {@link IOManager}.
*/
IOManager getIOManager();
/**
* Returns the current {@link MemoryManager}.
- *
+ *
* @return the current {@link MemoryManager}.
*/
MemoryManager getMemoryManager();
/**
* Returns the name of the task running in this environment.
- *
+ *
* @return the name of the task running in this environment
*/
String getTaskName();
/**
* Returns the next unbound input gate ID or <code>null</code> if no such ID exists
- *
+ *
* @return the next unbound input gate ID or <code>null</code> if no such ID exists
*/
GateID getNextUnboundInputGateID();
/**
* Returns the number of output gates registered with this environment.
- *
+ *
* @return the number of output gates registered with this environment
*/
int getNumberOfOutputGates();
/**
* Returns the number of input gates registered with this environment.
- *
+ *
* @return the number of input gates registered with this environment
*/
int getNumberOfInputGates();
/**
* Returns the number of output channels attached to this environment.
- *
+ *
* @return the number of output channels attached to this environment
*/
int getNumberOfOutputChannels();
/**
* Returns the number of input channels attached to this environment.
- *
+ *
* @return the number of input channels attached to this environment
*/
int getNumberOfInputChannels();
@@ -164,50 +166,48 @@ public interface Environment {
/**
* Returns the IDs of all output channels connected to this environment.
- *
+ *
* @return the IDs of all output channels connected to this environment
*/
Set<ChannelID> getOutputChannelIDs();
/**
* Returns the IDs of all input channels connected to this environment.
- *
+ *
* @return the IDs of all input channels connected to this environment
*/
Set<ChannelID> getInputChannelIDs();
/**
* Returns the IDs of all output gates connected to this environment.
- *
+ *
* @return the IDs of all output gates connected to this environment
*/
Set<GateID> getOutputGateIDs();
/**
* Returns the IDs of all input gates connected to this environment.
- *
+ *
* @return the IDs of all input gates connected to this environment
*/
Set<GateID> getInputGateIDs();
/**
* Returns the IDs of all the output channels connected to the gate with the given ID.
- *
- * @param gateID
- * the gate ID
+ *
+ * @param gateID the gate ID
* @return the IDs of all the output channels connected to the gate with the given ID
*/
Set<ChannelID> getOutputChannelIDsOfGate(GateID gateID);
/**
* Returns the IDs of all the input channels connected to the gate with the given ID.
- *
- * @param gateID
- * the gate ID
+ *
+ * @param gateID the gate ID
* @return the IDs of all the input channels connected to the gate with the given ID
*/
Set<ChannelID> getInputChannelIDsOfGate(GateID gateID);
-
+
/**
* Returns the proxy object for the accumulator protocol.
*/
@@ -215,11 +215,13 @@ public interface Environment {
/**
* Returns the buffer provider for this environment.
- * <p>
+ * <p/>
* 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
* @see eu.stratosphere.runtime.io.api.RecordWriter
*/
BufferProvider getOutputBufferProvider();
+
+ Map<String, FutureTask<Path>> getCopyTask();
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
index 55f036a..87674c8 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/ExecutionStateTransition.java
@@ -14,6 +14,8 @@
package eu.stratosphere.nephele.execution;
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.LogFactory;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
index 59787d2..29d6853 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
@@ -13,30 +13,11 @@
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.core.fs.Path;
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.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.nephele.protocols.AccumulatorProtocol;
import eu.stratosphere.nephele.services.iomanager.IOManager;
@@ -45,7 +26,6 @@ import eu.stratosphere.nephele.template.AbstractInvokable;
import eu.stratosphere.nephele.template.InputSplitProvider;
import eu.stratosphere.runtime.io.Buffer;
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.gates.GateID;
import eu.stratosphere.runtime.io.gates.InputGate;
@@ -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.LocalBufferPoolOwner;
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
* important services to the task. It keeps track of setting up the communication channels and provides access to input
* splits, memory manager, etc.
- * <p>
+ * <p/>
* This class is thread-safe.
*/
public class RuntimeEnvironment implements Environment, BufferProvider, LocalBufferPoolOwner, Runnable {
@@ -141,7 +136,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
* The observer object for the task's execution.
*/
private volatile ExecutionObserver executionObserver = null;
-
+
/**
* The RPC proxy to report accumulators to JobManager
*/
@@ -164,26 +159,22 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
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
* task.
- *
- * @param jobID
- * the ID of the original Nephele job
- * @param taskName
- * the name of task running in this environment
- * @param invokableClass
- * invokableClass the class that should be instantiated as a Nephele task
- * @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
+ *
+ * @param jobID the ID of the original Nephele job
+ * @param taskName the name of task running in this environment
+ * @param invokableClass invokableClass the class that should be instantiated as a Nephele task
+ * @param taskConfiguration the configuration object which was attached to the original JobVertex
+ * @param jobConfiguration the configuration object which was attached to the original JobGraph
+ * @throws Exception thrown if an error occurs while instantiating the invokable class
*/
public RuntimeEnvironment(final JobID jobID, final String taskName,
- final Class<? extends AbstractInvokable> invokableClass, final Configuration taskConfiguration,
- final Configuration jobConfiguration) throws Exception {
+ final Class<? extends AbstractInvokable> invokableClass, final Configuration taskConfiguration,
+ final Configuration jobConfiguration) throws Exception {
this.jobID = jobID;
this.taskName = taskName;
@@ -203,23 +194,18 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
/**
* Constructs a runtime environment from a task deployment description.
- *
- * @param tdd
- * the task deployment description
- * @param memoryManager
- * the task manager's memory manager component
- * @param ioManager
- * the task manager's I/O manager component
- * @param inputSplitProvider
- * the input split provider for this environment
- * @throws Exception
- * thrown if an error occurs while instantiating the invokable class
- */
- @SuppressWarnings({ "unchecked", "rawtypes" })
+ *
+ * @param tdd the task deployment description
+ * @param memoryManager the task manager's memory manager component
+ * @param ioManager the task manager's I/O manager component
+ * @param inputSplitProvider 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,
- final MemoryManager memoryManager, final IOManager ioManager,
- final InputSplitProvider inputSplitProvider,
- AccumulatorProtocol accumulatorProtocolProxy, Map<String, FutureTask<Path>> cpTasks) throws Exception {
+ final MemoryManager memoryManager, final IOManager ioManager,
+ final InputSplitProvider inputSplitProvider,
+ AccumulatorProtocol accumulatorProtocolProxy, Map<String, FutureTask<Path>> cpTasks) throws Exception {
this.jobID = tdd.getJobID();
this.taskName = tdd.getTaskName();
@@ -246,14 +232,14 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
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));
}
}
/**
* Returns the invokable object that represents the Nephele task.
- *
+ *
* @return the invokable object that represents the Nephele task
*/
public AbstractInvokable getInvokable() {
@@ -272,7 +258,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
@Override
public OutputGate createAndRegisterOutputGate() {
- OutputGate gate = new OutputGate(getJobID(), new GateID(), getNumberOfOutputGates());
+ OutputGate gate = new OutputGate(getJobID(), new GateID(), getNumberOfOutputGates());
this.outputGates.add(gate);
return gate;
@@ -318,7 +304,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
if (this.executionObserver.isCanceled() || t instanceof CancelTaskException) {
changeExecutionState(ExecutionState.CANCELED, null);
- } else {
+ }
+ else {
changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(t));
}
@@ -347,7 +334,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
if (this.executionObserver.isCanceled() || t instanceof CancelTaskException) {
changeExecutionState(ExecutionState.CANCELED, null);
- } else {
+ }
+ else {
changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(t));
}
@@ -400,9 +388,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
/**
* Returns the registered input gate with index <code>pos</code>.
- *
- * @param pos
- * the index of the input gate to return
+ *
+ * @param pos 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
*/
public InputGate<? extends IOReadableWritable> getInputGate(final int pos) {
@@ -415,9 +402,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
/**
* Returns the registered output gate with index <code>pos</code>.
- *
- * @param index
- * the index of the output gate to return
+ *
+ * @param index 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
*/
public OutputGate getOutputGate(int index) {
@@ -430,7 +416,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
/**
* Returns the thread which is assigned to execute the user code.
- *
+ *
* @return the thread which is assigned to execute the user code
*/
public Thread getExecutingThread() {
@@ -439,7 +425,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
if (this.executingThread == null) {
if (this.taskName == null) {
this.executingThread = new Thread(this);
- } else {
+ }
+ else {
this.executingThread = new Thread(this, getTaskNameWithIndex());
}
}
@@ -450,11 +437,9 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
/**
* Blocks until all output channels are closed.
- *
- * @throws IOException
- * 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 IOException 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
*/
private void waitForOutputChannelsToBeClosed() throws InterruptedException {
// Make sure, we leave this method with an InterruptedException when the task has been canceled
@@ -469,11 +454,9 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
/**
* Blocks until all input channels are closed.
- *
- * @throws IOException
- * 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 IOException 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
*/
private void waitForInputChannelsToBeClosed() throws IOException, InterruptedException {
// Wait for disconnection of all output gates
@@ -494,7 +477,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
if (allClosed) {
break;
- } else {
+ }
+ else {
Thread.sleep(SLEEPINTERVAL);
}
}
@@ -564,7 +548,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
/**
* Returns the name of the task with its index in the subtask group and the total number of subtasks.
- *
+ *
* @return the name of the task with its index in the subtask group and the total number of subtasks
*/
public String getTaskNameWithIndex() {
@@ -573,9 +557,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
/**
* Sets the execution observer for this environment.
- *
- * @param executionObserver
- * the execution observer for this environment
+ *
+ * @param executionObserver the execution observer for this environment
*/
public void setExecutionObserver(final ExecutionObserver executionObserver) {
this.executionObserver = executionObserver;
@@ -616,7 +599,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
@Override
public Set<ChannelID> getOutputChannelIDs() {
- Set<ChannelID> ids= new HashSet<ChannelID>();
+ Set<ChannelID> ids = new HashSet<ChannelID>();
for (OutputGate gate : this.outputGates) {
for (OutputChannel channel : gate.channels()) {
@@ -726,38 +709,47 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
public List<InputGate<? extends IOReadableWritable>> inputGates() {
return this.inputGates;
}
-
+
@Override
public AccumulatorProtocol getAccumulatorProtocolProxy() {
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
public BufferProvider getOutputBufferProvider() {
return this;
}
-
+
// -----------------------------------------------------------------------------------------------------------------
// BufferProvider methods
// -----------------------------------------------------------------------------------------------------------------
-
+
@Override
public Buffer requestBuffer(int minBufferSize) throws IOException {
return this.outputBufferPool.requestBuffer(minBufferSize);
}
-
+
@Override
public Buffer requestBufferBlocking(int minBufferSize) throws IOException, InterruptedException {
return this.outputBufferPool.requestBufferBlocking(minBufferSize);
}
-
+
@Override
public int getBufferSize() {
return this.outputBufferPool.getBufferSize();
}
-
+
@Override
- public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+ public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
return this.outputBufferPool.registerBufferAvailabilityListener(listener);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java
index 57ff073..8e9395a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/executiongraph/ExecutionVertex.java
@@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
+import eu.stratosphere.nephele.taskmanager.TaskKillResult;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -691,6 +692,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
* from the instance it is currently running on. If the task
* is not currently running, its execution state is simply
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java
index 50e0e7f..56b4eae 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/AbstractInstance.java
@@ -24,6 +24,7 @@ import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse;
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate;
import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
+import eu.stratosphere.nephele.taskmanager.TaskKillResult;
import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.nephele.ipc.RPC;
import eu.stratosphere.nephele.jobgraph.JobID;
@@ -205,6 +206,21 @@ public abstract class AbstractInstance extends NetworkNode {
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
public boolean equals(final Object obj) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
index 2ec2ed6..e32df61 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
@@ -13,11 +13,10 @@
package eu.stratosphere.nephele.jobgraph;
-import java.nio.ByteBuffer;
+import eu.stratosphere.nephele.AbstractID;
import javax.xml.bind.DatatypeConverter;
-
-import eu.stratosphere.nephele.AbstractID;
+import java.nio.ByteBuffer;
public final class JobID extends AbstractID {
@@ -44,6 +43,12 @@ public final class JobID extends AbstractID {
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) {
long lower = buf.getLong(offset);
long upper = buf.getLong(offset + 8);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
index 846ca2e..3ae9f3b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
@@ -31,6 +31,8 @@ import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
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.CommandLineParser;
import org.apache.commons.cli.GnuParser;
@@ -893,6 +895,38 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
}
@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
public void killInstance(final StringRecord instanceName) throws IOException {
final AbstractInstance instance = this.instanceManager.getInstanceByName(instanceName.toString());
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java
index 762b494..e369613 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java
@@ -21,6 +21,7 @@ import java.util.Map;
import java.util.Queue;
import java.util.Set;
+import eu.stratosphere.nephele.taskmanager.AbstractTaskResult;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -87,8 +88,8 @@ public final class RecoveryLogic {
verticesToBeRestarted.put(vertex.getID(), vertex);
final TaskCancelResult cancelResult = vertex.cancelTask();
- if (cancelResult.getReturnCode() != ReturnCode.SUCCESS
- && cancelResult.getReturnCode() != ReturnCode.TASK_NOT_FOUND) {
+ if (cancelResult.getReturnCode() != AbstractTaskResult.ReturnCode.SUCCESS
+ && cancelResult.getReturnCode() != AbstractTaskResult.ReturnCode.TASK_NOT_FOUND) {
verticesToBeRestarted.remove(vertex.getID());
LOG.error("Unable to cancel vertex" + cancelResult.getDescription());
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ExtendedManagementProtocol.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ExtendedManagementProtocol.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ExtendedManagementProtocol.java
index 461c797..59ec15d 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ExtendedManagementProtocol.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ExtendedManagementProtocol.java
@@ -24,6 +24,7 @@ import eu.stratosphere.nephele.instance.InstanceType;
import eu.stratosphere.nephele.instance.InstanceTypeDescription;
import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.nephele.managementgraph.ManagementGraph;
+import eu.stratosphere.nephele.managementgraph.ManagementVertexID;
import eu.stratosphere.nephele.topology.NetworkTopology;
/**
@@ -81,6 +82,18 @@ public interface ExtendedManagementProtocol extends JobManagementProtocol {
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).
*
* @param instanceName
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
index 19522db..93d8cdf 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
@@ -23,6 +23,7 @@ import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse;
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate;
import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
+import eu.stratosphere.nephele.taskmanager.TaskKillResult;
import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.nephele.taskmanager.TaskCancelResult;
import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult;
@@ -58,6 +59,17 @@ public interface TaskOperationProtocol extends VersionedProtocol {
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}
* object.
*
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java
index 06eec0c..825eae1 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java
@@ -25,6 +25,7 @@ import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.nephele.profiling.TaskManagerProfiler;
import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
import eu.stratosphere.nephele.template.AbstractInvokable;
+import eu.stratosphere.util.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -57,9 +58,7 @@ public final class Task implements ExecutionObserver {
private Queue<ExecutionListener> registeredListeners = new ConcurrentLinkedQueue<ExecutionListener>();
- public Task(final ExecutionVertexID vertexID, final RuntimeEnvironment environment,
- final TaskManager taskManager) {
-
+ public Task(ExecutionVertexID vertexID, final RuntimeEnvironment environment, TaskManager taskManager) {
this.vertexID = vertexID;
this.environment = environment;
this.taskManager = taskManager;
@@ -102,58 +101,51 @@ public final class Task implements ExecutionObserver {
executionStateChanged(ExecutionState.FAILED, "Execution thread died unexpectedly");
}
- /**
- * 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;
+ public void cancelExecution() {
+ cancelOrKillExecution(true);
}
- /**
- * Starts the execution of this task.
- */
- public void startExecution() {
-
- final Thread thread = this.environment.getExecutingThread();
- thread.start();
+ public void killExecution() {
+ cancelOrKillExecution(false);
}
/**
- * 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();
if (executingThread == null) {
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());
- this.isCanceled = true;
- // Change state
- executionStateChanged(ExecutionState.CANCELING, null);
+ if (cancel) {
+ this.isCanceled = true;
+ // Change state
+ executionStateChanged(ExecutionState.CANCELING, null);
- // Request user code to shut down
- try {
- final AbstractInvokable invokable = this.environment.getInvokable();
- if (invokable != null) {
- invokable.cancel();
+ // Request user code to shut down
+ try {
+ final AbstractInvokable invokable = this.environment.getInvokable();
+ if (invokable != null) {
+ invokable.cancel();
+ }
+ } catch (Throwable e) {
+ LOG.error(StringUtils.stringifyException(e));
}
- } catch (Throwable e) {
- LOG.error("Error while canceling task", e);
}
// Continuously interrupt the user thread until it changed to state CANCELED
while (true) {
+
executingThread.interrupt();
if (!executingThread.isAlive()) {
@@ -168,12 +160,36 @@ public final class Task implements ExecutionObserver {
break;
}
- if (LOG.isDebugEnabled())
- LOG.debug("Sending repeated canceling signal to " +
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Sending repeated " + (cancel == true ? "canceling" : "killing") + " signal to " +
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.
@@ -324,5 +340,4 @@ public final class Task implements ExecutionObserver {
return this.environment;
}
-
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskKillResult.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskKillResult.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskKillResult.java
new file mode 100644
index 0000000..3c0002c
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskKillResult.java
@@ -0,0 +1,44 @@
+/***********************************************************************************************************************
+ * 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;
+
+import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
+
+/**
+ * A <code>TaskKillResult</code> is used to report the results
+ * 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 class TaskKillResult extends AbstractTaskResult {
+
+ /**
+ * Constructs a new task kill result.
+ *
+ * @param vertexID
+ * the task ID this result belongs to
+ * @param returnCode
+ * the return code of the kill
+ */
+ public TaskKillResult(final ExecutionVertexID vertexID, final ReturnCode returnCode) {
+ super(vertexID, returnCode);
+ }
+
+ /**
+ * Constructs an empty task kill result.
+ */
+ public TaskKillResult() {
+ super();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
index 5240fc8..3b478cf 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
@@ -126,7 +126,7 @@ public class TaskManager implements TaskOperationProtocol {
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.
*/
private final ChannelManager channelManager;
@@ -286,16 +286,32 @@ public class TaskManager implements TaskOperationProtocol {
ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE);
- // Initialize the byte buffered channel manager
- ChannelManager channelManager = null;
+ int numInThreads = GlobalConfiguration.getInteger(
+ 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 {
- 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) {
LOG.error(StringUtils.stringifyException(ioe));
throw new Exception("Failed to instantiate Byte-buffered channel manager. " + ioe.getMessage(), ioe);
}
- this.channelManager = channelManager;
-
+
{
HardwareDescription resources = HardwareDescriptionFactory.extractFromSystem();
@@ -933,7 +949,7 @@ public class TaskManager implements TaskOperationProtocol {
}
@Override
- public void logBufferUtilization() throws IOException {
+ public void logBufferUtilization() {
this.channelManager.logBufferUtilization();
}
@@ -956,7 +972,7 @@ public class TaskManager implements TaskOperationProtocol {
@Override
public void invalidateLookupCacheEntries(final Set<ChannelID> channelIDs) throws IOException {
- this.byteBufferedChannelManager.invalidateLookupCacheEntries(channelIDs);
+ this.channelManager.invalidateLookupCacheEntries(channelIDs);
}
/**
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ExecutorThreadFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ExecutorThreadFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ExecutorThreadFactory.java
new file mode 100644
index 0000000..ed845e8
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ExecutorThreadFactory.java
@@ -0,0 +1,35 @@
+/***********************************************************************************************************************
+ * 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;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/AbstractIterativePactTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/AbstractIterativePactTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/AbstractIterativePactTask.java
index 674f270..79b9c83 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/AbstractIterativePactTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/AbstractIterativePactTask.java
@@ -21,7 +21,6 @@ import eu.stratosphere.api.common.typeutils.TypeSerializer;
import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
import eu.stratosphere.core.memory.DataOutputView;
import eu.stratosphere.nephele.execution.Environment;
-import eu.stratosphere.nephele.io.MutableReader;
import eu.stratosphere.pact.runtime.hash.CompactingHashTable;
import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannel;
import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannelBroker;
@@ -36,6 +35,7 @@ import eu.stratosphere.pact.runtime.task.RegularPactTask;
import eu.stratosphere.pact.runtime.task.ResettablePactDriver;
import eu.stratosphere.pact.runtime.task.util.TaskConfig;
import eu.stratosphere.pact.runtime.udf.RuntimeUDFContext;
+import eu.stratosphere.runtime.io.api.MutableReader;
import eu.stratosphere.types.Value;
import eu.stratosphere.util.Collector;
import eu.stratosphere.util.InstantiationUtil;
@@ -322,14 +322,15 @@ public abstract class AbstractIterativePactTask<S extends Function, OT> extends
* <p/>
* 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)
- * {@link SolutionSetFastUpdateOutputCollector} or normal (re-probing) {@link SolutionSetUpdateOutputCollector}
- * is created.
+ * {@link eu.stratosphere.pact.runtime.iterative.io.SolutionSetFastUpdateOutputCollector} or normal (re-probing)
+ * {@link SolutionSetUpdateOutputCollector} is created.
* <p/>
* 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.
*
* @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) {
Broker<CompactingHashTable<?>> solutionSetBroker = SolutionSetBroker.instance();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationHeadPactTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationHeadPactTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationHeadPactTask.java
index c39e3ef..89571c4 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationHeadPactTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/iterative/task/IterationHeadPactTask.java
@@ -17,6 +17,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
+import eu.stratosphere.runtime.io.api.BufferWriter;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -27,8 +28,6 @@ import eu.stratosphere.api.common.typeutils.TypeSerializer;
import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
import eu.stratosphere.core.memory.DataInputView;
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.io.InputViewIterator;
import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannel;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
index 46e3249..6491749 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/OutputEmitter.java
@@ -18,7 +18,6 @@ import eu.stratosphere.api.common.typeutils.TypeComparator;
import eu.stratosphere.runtime.io.api.ChannelSelector;
import eu.stratosphere.pact.runtime.plugable.SerializationDelegate;
-
public class OutputEmitter<T> implements ChannelSelector<SerializationDelegate<T>> {
private final ShipStrategyType strategy; // the shipping strategy used by this output emitter
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
index ba352eb..7fe35b4 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/shipping/RecordOutputEmitter.java
@@ -15,13 +15,10 @@ package eu.stratosphere.pact.runtime.shipping;
import eu.stratosphere.api.common.distributions.DataDistribution;
import eu.stratosphere.api.common.typeutils.TypeComparator;
-import eu.stratosphere.nephele.io.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.Record;
-
public class RecordOutputEmitter implements ChannelSelector<Record> {
// ------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
index cb3e782..638a7aa 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/DataSinkTask.java
@@ -193,8 +193,9 @@ public class DataSinkTask<IT> extends AbstractOutputTask {
}
// drop, if the task was canceled
else if (!this.taskCanceled) {
- if (LOG.isErrorEnabled())
+ if (LOG.isErrorEnabled()) {
LOG.error(getLogString("Error in user code: " + ex.getMessage()), ex);
+ }
throw ex;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
index b01799a..92c4648 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/RegularPactTask.java
@@ -22,9 +22,7 @@ import eu.stratosphere.api.common.typeutils.TypeComparator;
import eu.stratosphere.api.common.typeutils.TypeComparatorFactory;
import eu.stratosphere.api.common.typeutils.TypeSerializer;
import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
-import eu.stratosphere.configuration.ConfigConstants;
import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.configuration.GlobalConfiguration;
import eu.stratosphere.core.io.IOReadableWritable;
import eu.stratosphere.nephele.execution.CancelTaskException;
import eu.stratosphere.nephele.execution.Environment;
@@ -280,8 +278,9 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
@Override
public void invoke() throws Exception {
- if (LOG.isDebugEnabled())
+ if (LOG.isDebugEnabled()) {
LOG.debug(formatLogString("Start task code."));
+ }
// 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,
@@ -390,8 +389,9 @@ public class RegularPactTask<S extends Function, OT> extends AbstractTask implem
public void cancel() throws Exception {
this.running = false;
- if (LOG.isDebugEnabled())
+ if (LOG.isDebugEnabled()) {
LOG.debug(formatLogString("Cancelling task code"));
+ }
try {
if (this.driver != null) {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java
index c192cb9..f3f51f1 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/Buffer.java
@@ -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
*/
private Buffer(Buffer toDuplicate) {
@@ -74,7 +72,8 @@ public class Buffer {
}
public void recycleBuffer() {
- if (this.referenceCounter.decrementAndGet() == 0) {
+ int refCount = this.referenceCounter.decrementAndGet();
+ if (refCount == 0) {
this.recycler.recycle(this.memorySegment);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java
index dfa5d5e..17fff02 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java
@@ -17,8 +17,8 @@ import java.io.IOException;
import eu.stratosphere.nephele.event.task.AbstractEvent;
import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.runtime.io.network.envelope.Envelope;
-import eu.stratosphere.runtime.io.network.envelope.EnvelopeDispatcher;
+import eu.stratosphere.runtime.io.network.Envelope;
+import eu.stratosphere.runtime.io.network.EnvelopeDispatcher;
/**
* The base class for channel objects.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java
index 66be7de..ba78c01 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java
@@ -13,10 +13,10 @@
package eu.stratosphere.runtime.io.channels;
-import java.nio.ByteBuffer;
-
import eu.stratosphere.nephele.AbstractID;
+import java.nio.ByteBuffer;
+
public class ChannelID extends AbstractID {
public ChannelID() {
@@ -31,6 +31,12 @@ public class ChannelID extends AbstractID {
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) {
long lower = buf.getLong(offset);
long upper = buf.getLong(offset + 8);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java
index 5d5b53d..3007489 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java
@@ -24,3 +24,8 @@ public enum ChannelType {
/** In-memory channels */
IN_MEMORY
}
+
+
+
+
+
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/InputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/InputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/InputChannel.java
index 860141d..6122c36 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/InputChannel.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/InputChannel.java
@@ -21,7 +21,7 @@ import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.gates.InputChannelResult;
import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
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.serialization.AdaptiveSpanningRecordDeserializer;
import eu.stratosphere.runtime.io.serialization.RecordDeserializer;
@@ -30,7 +30,6 @@ import eu.stratosphere.runtime.io.serialization.RecordDeserializer.Deserializati
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import java.io.EOFException;
import java.io.IOException;
import java.util.ArrayDeque;
import java.util.Arrays;
@@ -83,6 +82,8 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
private int lastReceivedEnvelope = -1;
+ private ChannelID lastSourceID = null;
+
private boolean destroyCalled = false;
// ----------------------
@@ -104,7 +105,7 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
* the ID of the channel this channel is connected to
*/
public InputChannel(final InputGate<T> inputGate, final int channelIndex, final ChannelID channelID,
- final ChannelID connectedChannelID, ChannelType type) {
+ final ChannelID connectedChannelID, ChannelType type) {
super(channelIndex, channelID, connectedChannelID, type);
this.inputGate = inputGate;
this.deserializer = new AdaptiveSpanningRecordDeserializer<T>();
@@ -158,6 +159,9 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
// public abstract AbstractTaskEvent getCurrentEvent();
+ private DeserializationResult lastDeserializationResult;
+
+
public InputChannelResult readRecord(T target) throws IOException {
if (this.dataBuffer == null) {
if (isClosed()) {
@@ -176,7 +180,7 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
{
// sanity check: an event may only come after a complete record.
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();
@@ -202,8 +206,8 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
}
}
-
DeserializationResult deserializationResult = this.deserializer.getNextRecord(target);
+ this.lastDeserializationResult = deserializationResult;
if (deserializationResult.isBufferConsumed()) {
releasedConsumedReadBuffer(this.dataBuffer);
@@ -348,6 +352,7 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
this.queuedEnvelopes.add(envelope);
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)
if (envelope.getBuffer() != null) {
@@ -432,7 +437,7 @@ public class InputChannel<T extends IOReadableWritable> extends Channel implemen
}
@Override
- public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+ public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
return this.inputGate.registerBufferAvailabilityListener(listener);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/OutputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/OutputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/OutputChannel.java
index 7ca916c..f4fed65 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/OutputChannel.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/OutputChannel.java
@@ -17,7 +17,7 @@ import eu.stratosphere.nephele.event.task.AbstractEvent;
import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
import eu.stratosphere.nephele.jobgraph.JobID;
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 org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -107,7 +107,6 @@ public class OutputChannel extends Channel {
this.receiverCloseRequested = true;
this.closeLock.notifyAll();
}
- LOG.debug("OutputChannel received close event from target.");
}
else if (event instanceof AbstractTaskEvent) {
if (LOG.isDebugEnabled()) {
@@ -165,7 +164,7 @@ public class OutputChannel extends Channel {
private void checkStatus() throws IOException {
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) {
throw new ReceiverAlreadyClosedException();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputGate.java
index bdac7a2..c623220 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputGate.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputGate.java
@@ -14,9 +14,6 @@
package eu.stratosphere.runtime.io.gates;
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.LinkedBlockingQueue;
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.GateDeploymentDescriptor;
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.BufferProvider;
import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool;
@@ -38,7 +34,6 @@ import eu.stratosphere.nephele.event.task.AbstractEvent;
import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
import eu.stratosphere.nephele.execution.Environment;
import eu.stratosphere.runtime.io.channels.InputChannel;
-import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.nephele.jobgraph.JobID;
/**
@@ -378,7 +373,7 @@ public class InputGate<T extends IOReadableWritable> extends Gate<T> implements
}
@Override
- public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+ public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
return this.bufferPool.registerBufferAvailabilityListener(listener);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java
index d3eaea1..bff9180 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java
@@ -19,7 +19,6 @@ import eu.stratosphere.nephele.deployment.GateDeploymentDescriptor;
import eu.stratosphere.nephele.event.task.AbstractEvent;
import eu.stratosphere.runtime.io.Buffer;
import eu.stratosphere.runtime.io.channels.ChannelID;
-import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.runtime.io.channels.OutputChannel;
import eu.stratosphere.nephele.jobgraph.JobID;
[15/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractSingleGateRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractSingleGateRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractSingleGateRecordReader.java
new file mode 100644
index 0000000..71d5628
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractSingleGateRecordReader.java
@@ -0,0 +1,69 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.runtime.io.gates.InputGate;
+import eu.stratosphere.nephele.template.AbstractInvokable;
+
+import java.io.IOException;
+
+/**
+ * This is an abstract base class for a record reader, either dealing with mutable or immutable records.
+ *
+ * @param <T> The type of the record that can be read from this record reader.
+ */
+public abstract class AbstractSingleGateRecordReader<T extends IOReadableWritable> extends AbstractRecordReader {
+
+ /**
+ * The input gate associated with the record reader.
+ */
+ protected final InputGate<T> inputGate;
+
+ // --------------------------------------------------------------------------------------------
+
+ protected AbstractSingleGateRecordReader(AbstractInvokable invokable) {
+ this.inputGate = invokable.getEnvironment().createAndRegisterInputGate();
+ }
+
+ /**
+ * Returns the number of input channels wired to this reader's input gate.
+ *
+ * @return the number of input channels wired to this reader's input gate
+ */
+ public int getNumberOfInputChannels() {
+ return this.inputGate.getNumberOfInputChannels();
+ }
+
+ /**
+ * Publishes an event.
+ *
+ * @param event
+ * the event to be published
+ * @throws IOException
+ * thrown if an error occurs while transmitting the event
+ * @throws InterruptedException
+ * thrown if the thread is interrupted while waiting for the event to be published
+ */
+ @Override
+ public void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException {
+ // Delegate call to input gate to send events
+ this.inputGate.publishEvent(event);
+ }
+
+ InputGate<T> getInputGate() {
+ return this.inputGate;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractUnionRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractUnionRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractUnionRecordReader.java
new file mode 100644
index 0000000..69ac327
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/AbstractUnionRecordReader.java
@@ -0,0 +1,155 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.runtime.io.gates.InputChannelResult;
+import eu.stratosphere.runtime.io.gates.RecordAvailabilityListener;
+import eu.stratosphere.runtime.io.gates.InputGate;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.HashSet;
+import java.util.Set;
+
+public abstract class AbstractUnionRecordReader<T extends IOReadableWritable> extends AbstractRecordReader implements RecordAvailabilityListener<T> {
+
+ /**
+ * The set of all input gates.
+ */
+ private final InputGate<T>[] allInputGates;
+
+ /**
+ * The set of unclosed input gates.
+ */
+ private final Set<InputGate<T>> remainingInputGates;
+
+ /**
+ * Queue with indices of channels that store at least one available record.
+ */
+ private final ArrayDeque<InputGate<T>> availableInputGates = new ArrayDeque<InputGate<T>>();
+
+ /**
+ * The next input gate to read a record from.
+ */
+ private InputGate<T> nextInputGateToReadFrom;
+
+
+ @Override
+ public boolean isInputClosed() {
+ return this.remainingInputGates.isEmpty();
+ }
+
+ /**
+ * Constructs a new mutable union record reader.
+ *
+ * @param recordReaders
+ * the individual mutable record readers whose input is used to construct the union
+ */
+ @SuppressWarnings("unchecked")
+ protected AbstractUnionRecordReader(MutableRecordReader<T>[] recordReaders) {
+
+ if (recordReaders == null) {
+ throw new IllegalArgumentException("Provided argument recordReaders is null");
+ }
+
+ if (recordReaders.length < 2) {
+ throw new IllegalArgumentException(
+ "The mutable union record reader must at least be initialized with two individual mutable record readers");
+ }
+
+ this.allInputGates = new InputGate[recordReaders.length];
+ this.remainingInputGates = new HashSet<InputGate<T>>((int) (recordReaders.length * 1.6f));
+
+ for (int i = 0; i < recordReaders.length; i++) {
+ InputGate<T> inputGate = recordReaders[i].getInputGate();
+ inputGate.registerRecordAvailabilityListener(this);
+ this.allInputGates[i] = inputGate;
+ this.remainingInputGates.add(inputGate);
+ }
+ }
+
+
+ @Override
+ public void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException {
+ for (InputGate<T> gate : this.allInputGates) {
+ gate.publishEvent(event);
+ }
+ }
+
+ @Override
+ public void reportRecordAvailability(InputGate<T> inputGate) {
+ synchronized (this.availableInputGates) {
+ this.availableInputGates.add(inputGate);
+ this.availableInputGates.notifyAll();
+ }
+ }
+
+ protected boolean getNextRecord(T target) throws IOException, InterruptedException {
+
+ while (true) {
+ // has the current input gate more data?
+ if (this.nextInputGateToReadFrom == null) {
+ if (this.remainingInputGates.isEmpty()) {
+ return false;
+ }
+
+ this.nextInputGateToReadFrom = getNextAvailableInputGate();
+ }
+
+ InputChannelResult result = this.nextInputGateToReadFrom.readRecord(target);
+ switch (result) {
+ case INTERMEDIATE_RECORD_FROM_BUFFER: // record is available and we can stay on the same channel
+ return true;
+
+ case LAST_RECORD_FROM_BUFFER: // record is available, but we need to re-check the channels
+ this.nextInputGateToReadFrom = null;
+ return true;
+
+ case END_OF_SUPERSTEP:
+ this.nextInputGateToReadFrom = null;
+ if (incrementEndOfSuperstepEventAndCheck()) {
+ return false; // end of the superstep
+ }
+ else {
+ break; // fall through and wait for next record/event
+ }
+
+ case TASK_EVENT: // event for the subscribers is available
+ handleEvent(this.nextInputGateToReadFrom.getCurrentEvent());
+ this.nextInputGateToReadFrom = null;
+ break;
+
+ case END_OF_STREAM: // one gate is empty
+ this.remainingInputGates.remove(this.nextInputGateToReadFrom);
+ this.nextInputGateToReadFrom = null;
+ break;
+
+ case NONE: // gate processed an internal event and could not return a record on this call
+ this.nextInputGateToReadFrom = null;
+ break;
+ }
+ }
+ }
+
+ private InputGate<T> getNextAvailableInputGate() throws InterruptedException {
+ synchronized (this.availableInputGates) {
+ while (this.availableInputGates.isEmpty()) {
+ this.availableInputGates.wait();
+ }
+ return this.availableInputGates.pop();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/BufferWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/BufferWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/BufferWriter.java
new file mode 100644
index 0000000..a7b62e0
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/BufferWriter.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.nephele.event.task.EventListener;
+import eu.stratosphere.runtime.io.channels.EndOfSuperstepEvent;
+import eu.stratosphere.nephele.template.AbstractInvokable;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.gates.OutputGate;
+
+import java.io.IOException;
+
+public class BufferWriter {
+
+ protected final OutputGate outputGate;
+
+ public BufferWriter(AbstractInvokable invokable) {
+ this.outputGate = invokable.getEnvironment().createAndRegisterOutputGate();
+ }
+
+ public void sendBuffer(Buffer buffer, int targetChannel) throws IOException, InterruptedException {
+ this.outputGate.sendBuffer(buffer, targetChannel);
+ }
+
+ public void sendEvent(AbstractEvent event, int targetChannel) throws IOException, InterruptedException {
+ this.outputGate.sendEvent(event, targetChannel);
+ }
+
+ public void sendBufferAndEvent(Buffer buffer, AbstractEvent event, int targetChannel) throws IOException, InterruptedException {
+ this.outputGate.sendBufferAndEvent(buffer, event, targetChannel);
+ }
+
+ public void broadcastBuffer(Buffer buffer) throws IOException, InterruptedException {
+ this.outputGate.broadcastBuffer(buffer);
+ }
+
+ public void broadcastEvent(AbstractEvent event) throws IOException, InterruptedException {
+ this.outputGate.broadcastEvent(event);
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ public void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
+ this.outputGate.subscribeToEvent(eventListener, eventType);
+ }
+
+ public void unsubscribeFromEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
+ this.outputGate.unsubscribeFromEvent(eventListener, eventType);
+ }
+
+ public void sendEndOfSuperstep() throws IOException, InterruptedException {
+ this.outputGate.broadcastEvent(EndOfSuperstepEvent.INSTANCE);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ChannelSelector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ChannelSelector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ChannelSelector.java
new file mode 100644
index 0000000..b9638ea
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ChannelSelector.java
@@ -0,0 +1,39 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+
+/**
+ * Objects implementing this interface are passed to an {@link eu.stratosphere.runtime.io.gates.OutputGate}. When a record is sent through the output
+ * gate, the channel selector object is called to determine to which {@link eu.stratosphere.runtime.io.channels.OutputChannel} objects the record
+ * shall be passed on.
+ *
+ * @param <T>
+ * the type of record which is sent through the attached output gate
+ */
+public interface ChannelSelector<T extends IOReadableWritable> {
+
+ /**
+ * Called to determine to which attached {@link eu.stratosphere.runtime.io.channels.OutputChannel} objects the given record shall be forwarded.
+ *
+ * @param record
+ * the record to the determine the output channels for
+ * @param numberOfOutputChannels
+ * the total number of output channels which are attached to respective output gate
+ * @return a (possibly empty) array of integer numbers which indicate the indices of the output channels through
+ * which the record shall be forwarded
+ */
+ int[] selectChannels(T record, int numberOfOutputChannels);
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableReader.java
new file mode 100644
index 0000000..ddc080f
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableReader.java
@@ -0,0 +1,32 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import java.io.IOException;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+
+/**
+ *
+ */
+public interface MutableReader<T extends IOReadableWritable> extends ReaderBase {
+
+ /**
+ * @param target
+ * @return
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ boolean next(T target) throws IOException, InterruptedException;
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableRecordReader.java
new file mode 100644
index 0000000..9d03c7f
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableRecordReader.java
@@ -0,0 +1,120 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import java.io.IOException;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.runtime.io.gates.InputChannelResult;
+import eu.stratosphere.nephele.template.AbstractOutputTask;
+import eu.stratosphere.nephele.template.AbstractTask;
+
+public class MutableRecordReader<T extends IOReadableWritable> extends AbstractSingleGateRecordReader<T> implements MutableReader<T> {
+
+ private boolean endOfStream;
+
+
+ /**
+ * Constructs a new mutable record reader and registers a new input gate with the application's environment.
+ *
+ * @param taskBase The application that instantiated the record reader.
+ */
+ public MutableRecordReader(final AbstractTask taskBase) {
+ super(taskBase);
+ }
+
+ /**
+ * Constructs a new record reader and registers a new input gate with the application's environment.
+ *
+ * @param outputBase The application that instantiated the record reader.
+ */
+ public MutableRecordReader(final AbstractOutputTask outputBase) {
+ super(outputBase);
+ }
+
+ /**
+ * Constructs a new record reader and registers a new input gate with the application's environment.
+ *
+ * @param taskBase
+ * the application that instantiated the record reader
+ * @param inputGateID
+ * The ID of the input gate that the reader reads from.
+ */
+ public MutableRecordReader(final AbstractTask taskBase, final int inputGateID) {
+ super(taskBase);
+ }
+
+ /**
+ * Constructs a new record reader and registers a new input gate with the application's environment.
+ *
+ * @param outputBase
+ * the application that instantiated the record reader
+ * @param inputGateID
+ * The ID of the input gate that the reader reads from.
+ */
+ public MutableRecordReader(final AbstractOutputTask outputBase, final int inputGateID) {
+ super(outputBase);
+ }
+
+ // --------------------------------------------------------------------------------------------
+
+ @Override
+ public boolean next(final T target) throws IOException, InterruptedException {
+ if (this.endOfStream) {
+ return false;
+
+ }
+ while (true) {
+ InputChannelResult result = this.inputGate.readRecord(target);
+ switch (result) {
+ case INTERMEDIATE_RECORD_FROM_BUFFER:
+ case LAST_RECORD_FROM_BUFFER:
+ return true;
+
+ case END_OF_SUPERSTEP:
+ if (incrementEndOfSuperstepEventAndCheck()) {
+ return false; // end of the superstep
+ }
+ else {
+ break; // fall through and wait for next record/event
+ }
+
+ case TASK_EVENT:
+ handleEvent(this.inputGate.getCurrentEvent());
+ break; // fall through to get next record
+
+ case END_OF_STREAM:
+ this.endOfStream = true;
+ return false;
+
+ default:
+ ; // fall through to get next record
+ }
+ }
+ }
+
+ @Override
+ public boolean isInputClosed() {
+ return this.endOfStream;
+ }
+
+ @Override
+ public void setIterative(int numEventsUntilEndOfSuperstep) {
+ // sanity check for debug purposes
+ if (numEventsUntilEndOfSuperstep != getNumberOfInputChannels()) {
+ throw new IllegalArgumentException("Number of events till end of superstep is different from the number of input channels.");
+ }
+ super.setIterative(numEventsUntilEndOfSuperstep);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableUnionRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableUnionRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableUnionRecordReader.java
new file mode 100644
index 0000000..e79c7bb
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/MutableUnionRecordReader.java
@@ -0,0 +1,37 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import java.io.IOException;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+
+public class MutableUnionRecordReader<T extends IOReadableWritable> extends AbstractUnionRecordReader<T> implements MutableReader<T> {
+
+
+ /**
+ * Constructs a new mutable union record reader.
+ *
+ * @param recordReaders
+ * the individual mutable record readers whose input is used to construct the union
+ */
+ public MutableUnionRecordReader(MutableRecordReader<T>[] recordReaders) {
+ super(recordReaders);
+ }
+
+ @Override
+ public boolean next(T target) throws IOException, InterruptedException {
+ return getNextRecord(target);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/Reader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/Reader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/Reader.java
new file mode 100644
index 0000000..ba0d3b6
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/Reader.java
@@ -0,0 +1,30 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import java.io.IOException;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+
+/**
+ * A reader interface to read records from an input.
+ *
+ * @param <T> The type of the record that can be emitted with this record writer
+ */
+public interface Reader<T extends IOReadableWritable> extends ReaderBase {
+
+ boolean hasNext() throws IOException, InterruptedException;
+
+ T next() throws IOException, InterruptedException;
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ReaderBase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ReaderBase.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ReaderBase.java
new file mode 100644
index 0000000..17f6b7d
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/ReaderBase.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import java.io.IOException;
+
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.nephele.event.task.EventListener;
+
+
+/**
+ *
+ */
+public interface ReaderBase {
+
+ boolean isInputClosed();
+
+ /**
+ * Subscribes the listener object to receive events of the given type.
+ *
+ * @param eventListener
+ * the listener object to register
+ * @param eventType
+ * the type of event to register the listener for
+ */
+ void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType);
+
+ /**
+ * Removes the subscription for events of the given type for the listener object.
+ *
+ * @param eventListener
+ * the listener object to cancel the subscription for
+ * @param eventType
+ * the type of the event to cancel the subscription for
+ */
+ void unsubscribeFromEvent(final EventListener eventListener, final Class<? extends AbstractTaskEvent> eventType);
+
+ /**
+ * Publishes an event.
+ *
+ * @param event
+ * the event to be published
+ * @throws IOException
+ * thrown if an error occurs while transmitting the event
+ * @throws InterruptedException
+ * thrown if the thread is interrupted while waiting for the event to be published
+ */
+ void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException;
+
+
+ void setIterative(int numEventsUntilEndOfSuperstep);
+
+
+ void startNextSuperstep();
+
+ boolean hasReachedEndOfSuperstep();
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordReader.java
new file mode 100644
index 0000000..bb6a580
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordReader.java
@@ -0,0 +1,154 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.template.AbstractOutputTask;
+import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.runtime.io.gates.InputChannelResult;
+
+import java.io.IOException;
+
+/**
+ * A record writer connects an input gate to an application. It allows the application
+ * query for incoming records and read them from input gate.
+ *
+ * @param <T> The type of the record that can be read from this record reader.
+ */
+public class RecordReader<T extends IOReadableWritable> extends AbstractSingleGateRecordReader<T> implements Reader<T> {
+
+ private final Class<T> recordType;
+
+ /**
+ * Stores the last read record.
+ */
+ private T lookahead;
+
+ /**
+ * Stores if more no more records will be received from the assigned input gate.
+ */
+ private boolean noMoreRecordsWillFollow;
+
+ // --------------------------------------------------------------------------------------------
+
+ /**
+ * Constructs a new record reader and registers a new input gate with the application's environment.
+ *
+ * @param taskBase
+ * The application that instantiated the record reader.
+ * @param recordType
+ * The class of records that can be read from the record reader.
+ */
+ public RecordReader(AbstractTask taskBase, Class<T> recordType) {
+ super(taskBase);
+ this.recordType = recordType;
+ }
+
+ /**
+ * Constructs a new record reader and registers a new input gate with the application's environment.
+ *
+ * @param outputBase
+ * The application that instantiated the record reader.
+ * @param recordType
+ * The class of records that can be read from the record reader.
+ */
+ public RecordReader(AbstractOutputTask outputBase, Class<T> recordType) {
+ super(outputBase);
+ this.recordType = recordType;
+ }
+
+ // --------------------------------------------------------------------------------------------
+
+ /**
+ * Checks if at least one more record can be read from the associated input gate. This method may block
+ * until the associated input gate is able to read the record from one of its input channels.
+ *
+ * @return <code>true</code>it at least one more record can be read from the associated input gate, otherwise
+ * <code>false</code>
+ */
+ @Override
+ public boolean hasNext() throws IOException, InterruptedException{
+ if (this.lookahead != null) {
+ return true;
+ } else {
+ if (this.noMoreRecordsWillFollow) {
+ return false;
+ }
+
+ T record = instantiateRecordType();
+
+ while (true) {
+ InputChannelResult result = this.inputGate.readRecord(record);
+ switch (result) {
+ case INTERMEDIATE_RECORD_FROM_BUFFER:
+ case LAST_RECORD_FROM_BUFFER:
+ this.lookahead = record;
+ return true;
+
+ case END_OF_SUPERSTEP:
+ if (incrementEndOfSuperstepEventAndCheck()) {
+ return false;
+ }
+ else {
+ break; // fall through and wait for next record/event
+ }
+
+ case TASK_EVENT:
+ handleEvent(this.inputGate.getCurrentEvent());
+ break;
+
+ case END_OF_STREAM:
+ this.noMoreRecordsWillFollow = true;
+ return false;
+
+ default:
+ ; // fall through the loop
+ }
+ }
+ }
+ }
+
+ /**
+ * Reads the current record from the associated input gate.
+ *
+ * @return the current record from the associated input gate.
+ * @throws IOException
+ * thrown if any error occurs while reading the record from the input gate
+ */
+ @Override
+ public T next() throws IOException, InterruptedException {
+ if (hasNext()) {
+ T tmp = this.lookahead;
+ this.lookahead = null;
+ return tmp;
+ } else {
+ return null;
+ }
+ }
+
+ @Override
+ public boolean isInputClosed() {
+ return this.noMoreRecordsWillFollow;
+ }
+
+ private T instantiateRecordType() {
+ try {
+ return this.recordType.newInstance();
+ } catch (InstantiationException e) {
+ throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
+ } catch (IllegalAccessException e) {
+ throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordWriter.java
new file mode 100644
index 0000000..132dc14
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RecordWriter.java
@@ -0,0 +1,151 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.nephele.template.AbstractInputTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
+import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.channels.EndOfSuperstepEvent;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
+import eu.stratosphere.runtime.io.serialization.RecordSerializer;
+import eu.stratosphere.runtime.io.serialization.SpanningRecordSerializer;
+
+import java.io.IOException;
+
+/**
+ * A record writer connects the application to an output gate. It allows the application
+ * of emit (send out) to the output gate. The output gate will then take care of distributing
+ * the emitted records among the output channels.
+ *
+ * @param <T>
+ * the type of the record that can be emitted with this record writer
+ */
+public class RecordWriter<T extends IOReadableWritable> extends BufferWriter {
+
+ private final BufferProvider bufferPool;
+
+ private final ChannelSelector<T> channelSelector;
+
+ private int numChannels;
+
+ /** RecordSerializer per outgoing channel */
+ private RecordSerializer<T>[] serializers;
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ public RecordWriter(AbstractTask task) {
+ this((AbstractInvokable) task, new RoundRobinChannelSelector<T>());
+ }
+
+ public RecordWriter(AbstractTask task, ChannelSelector<T> channelSelector) {
+ this((AbstractInvokable) task, channelSelector);
+ }
+
+ public RecordWriter(AbstractInputTask<?> task) {
+ this((AbstractInvokable) task, new RoundRobinChannelSelector<T>());
+ }
+
+ public RecordWriter(AbstractInputTask<?> task, ChannelSelector<T> channelSelector) {
+ this((AbstractInvokable) task, channelSelector);
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ private RecordWriter(AbstractInvokable invokable, ChannelSelector<T> channelSelector) {
+ // initialize the gate
+ super(invokable);
+
+ this.bufferPool = invokable.getEnvironment().getOutputBufferProvider();
+ this.channelSelector = channelSelector;
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ @SuppressWarnings("unchecked")
+ public void initializeSerializers() {
+ this.numChannels = this.outputGate.getNumChannels();
+ this.serializers = new RecordSerializer[numChannels];
+ for (int i = 0; i < this.numChannels; i++) {
+ this.serializers[i] = new SpanningRecordSerializer<T>();
+ }
+ }
+
+ public void emit(final T record) throws IOException, InterruptedException {
+ for (int targetChannel : this.channelSelector.selectChannels(record, this.numChannels)) {
+ // serialize with corresponding serializer and send full buffer
+ RecordSerializer<T> serializer = this.serializers[targetChannel];
+
+ RecordSerializer.SerializationResult result = serializer.addRecord(record);
+ while (result.isFullBuffer()) {
+ Buffer buffer = serializer.getCurrentBuffer();
+ if (buffer != null) {
+ sendBuffer(buffer, targetChannel);
+ }
+
+ buffer = this.bufferPool.requestBufferBlocking(this.bufferPool.getBufferSize());
+ result = serializer.setNextBuffer(buffer);
+ }
+ }
+ }
+
+ public void flush() throws IOException, InterruptedException {
+ for (int targetChannel = 0; targetChannel < this.numChannels; targetChannel++) {
+ RecordSerializer<T> serializer = this.serializers[targetChannel];
+
+ Buffer buffer = serializer.getCurrentBuffer();
+ if (buffer != null) {
+ sendBuffer(buffer, targetChannel);
+ }
+
+ serializer.clear();
+ }
+ }
+
+ @Override
+ public void broadcastEvent(AbstractEvent event) throws IOException, InterruptedException {
+ for (int targetChannel = 0; targetChannel < this.numChannels; targetChannel++) {
+ RecordSerializer<T> serializer = this.serializers[targetChannel];
+
+ Buffer buffer = serializer.getCurrentBuffer();
+ if (buffer == null) {
+ super.sendEvent(event, targetChannel);
+ } else {
+ super.sendBufferAndEvent(buffer, event, targetChannel);
+
+ buffer = this.bufferPool.requestBufferBlocking(this.bufferPool.getBufferSize());
+ serializer.setNextBuffer(buffer);
+ }
+ }
+ }
+
+ @Override
+ public void sendEndOfSuperstep() throws IOException, InterruptedException {
+ for (int targetChannel = 0; targetChannel < this.numChannels; targetChannel++) {
+ RecordSerializer<T> serializer = this.serializers[targetChannel];
+
+ Buffer buffer = serializer.getCurrentBuffer();
+ if (buffer == null) {
+ super.sendEvent(EndOfSuperstepEvent.INSTANCE, targetChannel);
+ } else {
+ super.sendBufferAndEvent(buffer, EndOfSuperstepEvent.INSTANCE, targetChannel);
+
+ buffer = this.bufferPool.requestBufferBlocking(this.bufferPool.getBufferSize());
+ serializer.setNextBuffer(buffer);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RoundRobinChannelSelector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RoundRobinChannelSelector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RoundRobinChannelSelector.java
new file mode 100644
index 0000000..aaa1506
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/RoundRobinChannelSelector.java
@@ -0,0 +1,47 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+
+/**
+ * This is the default implementation of the {@link ChannelSelector} interface. It represents a simple round-robin
+ * strategy, i.e. regardless of the record every attached exactly one output channel is selected at a time.
+
+ * @param <T>
+ * the type of record which is sent through the attached output gate
+ */
+public class RoundRobinChannelSelector<T extends IOReadableWritable> implements ChannelSelector<T> {
+
+ /**
+ * Stores the index of the channel to send the next record to.
+ */
+ private final int[] nextChannelToSendTo = new int[1];
+
+ /**
+ * Constructs a new default channel selector.
+ */
+ public RoundRobinChannelSelector() {
+ this.nextChannelToSendTo[0] = 0;
+ }
+
+
+ @Override
+ public int[] selectChannels(final T record, final int numberOfOutputChannels) {
+
+ this.nextChannelToSendTo[0] = (this.nextChannelToSendTo[0] + 1) % numberOfOutputChannels;
+
+ return this.nextChannelToSendTo;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/UnionRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/UnionRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/UnionRecordReader.java
new file mode 100644
index 0000000..85ce389
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/api/UnionRecordReader.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * 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.api;
+
+import java.io.IOException;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+
+public final class UnionRecordReader<T extends IOReadableWritable> extends AbstractUnionRecordReader<T> implements Reader<T> {
+
+ private final Class<T> recordType;
+
+ private T lookahead;
+
+
+ public UnionRecordReader(MutableRecordReader<T>[] recordReaders, Class<T> recordType) {
+ super(recordReaders);
+ this.recordType = recordType;
+ }
+
+ @Override
+ public boolean hasNext() throws IOException, InterruptedException {
+ if (this.lookahead != null) {
+ return true;
+ } else {
+ T record = instantiateRecordType();
+ if (getNextRecord(record)) {
+ this.lookahead = record;
+ return true;
+ } else {
+ return false;
+ }
+ }
+ }
+
+ @Override
+ public T next() throws IOException, InterruptedException {
+ if (hasNext()) {
+ T tmp = this.lookahead;
+ this.lookahead = null;
+ return tmp;
+ } else {
+ return null;
+ }
+ }
+
+ private T instantiateRecordType() {
+ try {
+ return this.recordType.newInstance();
+ } catch (InstantiationException e) {
+ throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
+ } catch (IllegalAccessException e) {
+ throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/BufferOrEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/BufferOrEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/BufferOrEvent.java
new file mode 100644
index 0000000..2d51d13
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/BufferOrEvent.java
@@ -0,0 +1,52 @@
+/***********************************************************************************************************************
+ * 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.channels;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.runtime.io.Buffer;
+
+/**
+ * Either type for {@link eu.stratosphere.runtime.io.Buffer} and {@link AbstractEvent}.
+ */
+public class BufferOrEvent {
+
+ private final Buffer buffer;
+
+ private final AbstractEvent event;
+
+ public BufferOrEvent(Buffer buffer) {
+ this.buffer = buffer;
+ this.event = null;
+ }
+
+ public BufferOrEvent(AbstractEvent event) {
+ this.buffer = null;
+ this.event = event;
+ }
+
+ public boolean isBuffer() {
+ return this.buffer != null;
+ }
+
+ public boolean isEvent() {
+ return this.event != null;
+ }
+
+ public Buffer getBuffer() {
+ return this.buffer;
+ }
+
+ public AbstractEvent getEvent() {
+ return this.event;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java
new file mode 100644
index 0000000..dfa5d5e
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/Channel.java
@@ -0,0 +1,97 @@
+/***********************************************************************************************************************
+ * 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.channels;
+
+import java.io.IOException;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.runtime.io.network.envelope.Envelope;
+import eu.stratosphere.runtime.io.network.envelope.EnvelopeDispatcher;
+
+/**
+ * The base class for channel objects.
+ * <p>
+ * Every channel has an index (at the corresponding gate), ID, and type. The connected channel is given by the ID of
+ * destination channel.
+ */
+public abstract class Channel {
+
+ private final ChannelID id;
+
+ private final ChannelID connectedId;
+
+ private final int index;
+
+ private final ChannelType type;
+
+ protected EnvelopeDispatcher envelopeDispatcher;
+
+ /**
+ * Auxiliary constructor for channels
+ *
+ * @param index the index of the channel in either the output or input gate
+ * @param id the ID of the channel
+ * @param connectedId the ID of the channel this channel is connected to
+ */
+ protected Channel(int index, ChannelID id, ChannelID connectedId, ChannelType type) {
+ this.index = index;
+ this.id = id;
+ this.connectedId = connectedId;
+ this.type = type;
+ }
+
+ public int getIndex() {
+ return this.index;
+ }
+
+ public ChannelID getID() {
+ return this.id;
+ }
+
+ public ChannelID getConnectedId() {
+ return this.connectedId;
+ }
+
+ public ChannelType getChannelType() {
+ return this.type;
+ }
+
+ /**
+ * Registers an EnvelopeDispatcher with this channel at runtime.
+ *
+ * @param envelopeDispatcher the envelope dispatcher to use for data transfers
+ */
+ public void registerEnvelopeDispatcher(EnvelopeDispatcher envelopeDispatcher) {
+ this.envelopeDispatcher = envelopeDispatcher;
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ public abstract JobID getJobID();
+
+ public abstract boolean isInputChannel();
+
+ public abstract boolean isClosed() throws IOException, InterruptedException;
+
+ public abstract void transferEvent(AbstractEvent event) throws IOException, InterruptedException;
+
+ public abstract void queueEnvelope(Envelope envelope);
+
+ // nothing to do for buffer oriented runtime => TODO remove with pending changes for input side
+ public abstract void releaseAllResources();
+
+ // nothing to do for buffer oriented runtime => TODO remove with pending changes for input side
+ public abstract void destroy();
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelCloseEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelCloseEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelCloseEvent.java
new file mode 100644
index 0000000..dcdcbb2
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelCloseEvent.java
@@ -0,0 +1,33 @@
+/***********************************************************************************************************************
+ * 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.channels;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+
+public final class ChannelCloseEvent extends AbstractEvent {
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ // Nothing to do here
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ // Nothing to do here
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java
new file mode 100644
index 0000000..66be7de
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelID.java
@@ -0,0 +1,39 @@
+/***********************************************************************************************************************
+ * 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.channels;
+
+import java.nio.ByteBuffer;
+
+import eu.stratosphere.nephele.AbstractID;
+
+public class ChannelID extends AbstractID {
+
+ public ChannelID() {
+ super();
+ }
+
+ public ChannelID(long lowerPart, long upperPart) {
+ super(lowerPart, upperPart);
+ }
+
+ public ChannelID(byte[] bytes) {
+ super(bytes);
+ }
+
+ public static ChannelID fromByteBuffer(ByteBuffer buf, int offset) {
+ long lower = buf.getLong(offset);
+ long upper = buf.getLong(offset + 8);
+ return new ChannelID(lower, upper);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java
new file mode 100644
index 0000000..5d5b53d
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelType.java
@@ -0,0 +1,26 @@
+/***********************************************************************************************************************
+ * 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.channels;
+
+/**
+ * An enumeration for the channel types.
+ */
+public enum ChannelType {
+
+ /** Network channels */
+ NETWORK,
+
+ /** In-memory channels */
+ IN_MEMORY
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/EndOfSuperstepEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/EndOfSuperstepEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/EndOfSuperstepEvent.java
new file mode 100644
index 0000000..e5f9589
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/EndOfSuperstepEvent.java
@@ -0,0 +1,34 @@
+/***********************************************************************************************************************
+ * 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.channels;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+
+/**
+ * Marks the end of a superstep of one particular iteration head
+ */
+public class EndOfSuperstepEvent extends AbstractEvent {
+
+ public static final EndOfSuperstepEvent INSTANCE = new EndOfSuperstepEvent();
+
+ @Override
+ public void write(DataOutput out) throws IOException {}
+
+ @Override
+ public void read(DataInput in) throws IOException {}
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/InputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/InputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/InputChannel.java
new file mode 100644
index 0000000..860141d
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/InputChannel.java
@@ -0,0 +1,493 @@
+/***********************************************************************************************************************
+ * 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.channels;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.gates.InputChannelResult;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferAvailabilityListener;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider;
+import eu.stratosphere.runtime.io.network.envelope.Envelope;
+import eu.stratosphere.runtime.io.gates.InputGate;
+import eu.stratosphere.runtime.io.serialization.AdaptiveSpanningRecordDeserializer;
+import eu.stratosphere.runtime.io.serialization.RecordDeserializer;
+import eu.stratosphere.runtime.io.serialization.RecordDeserializer.DeserializationResult;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Queue;
+
+/**
+ * InputChannel is an abstract base class to all different kinds of concrete
+ * input channels that can be used. Input channels are always parameterized to
+ * a specific type that can be transported through the channel.
+
+ * @param <T> The Type of the record that can be transported through the channel.
+ */
+public class InputChannel<T extends IOReadableWritable> extends Channel implements BufferProvider {
+
+ private final InputGate<T> inputGate;
+
+ /**
+ * The log object used to report warnings and errors.
+ */
+ private static final Log LOG = LogFactory.getLog(InputChannel.class);
+
+ /**
+ * The deserializer used to deserialize records.
+ */
+ private final RecordDeserializer<T> deserializer;
+
+ /**
+ * Buffer for the uncompressed (raw) data.
+ */
+ private Buffer dataBuffer;
+
+ private AbstractTaskEvent currentEvent;
+
+ /**
+ * The exception observed in this channel while processing the buffers. Checked and thrown
+ * per-buffer.
+ */
+ private volatile IOException ioException;
+
+ /**
+ * Stores the number of bytes read through this input channel since its instantiation.
+ */
+ private long amountOfDataTransmitted;
+
+ private volatile boolean brokerAggreedToCloseChannel;
+
+ // -------------------------------------------------------------------------------------------
+
+ private int lastReceivedEnvelope = -1;
+
+ private boolean destroyCalled = false;
+
+ // ----------------------
+
+ private Queue<Envelope> queuedEnvelopes = new ArrayDeque<Envelope>();
+
+ private Iterator<AbstractEvent> pendingEvents;
+
+ /**
+ * Constructs an input channel with a given input gate associated.
+ *
+ * @param inputGate
+ * the input gate this channel is connected to
+ * @param channelIndex
+ * the index of the channel in the input gate
+ * @param channelID
+ * the ID of the channel
+ * @param connectedChannelID
+ * the ID of the channel this channel is connected to
+ */
+ public InputChannel(final InputGate<T> inputGate, final int channelIndex, final ChannelID channelID,
+ final ChannelID connectedChannelID, ChannelType type) {
+ super(channelIndex, channelID, connectedChannelID, type);
+ this.inputGate = inputGate;
+ this.deserializer = new AdaptiveSpanningRecordDeserializer<T>();
+ }
+
+ /**
+ * Returns the input gate associated with the input channel.
+ *
+ * @return the input gate associated with the input channel.
+ */
+ public InputGate<T> getInputGate() {
+ return this.inputGate;
+ }
+
+ /**
+ * Reads a record from the input channel. If currently no record is available the method
+ * returns <code>null</code>. If the channel is closed (i.e. no more records will be received), the method
+ * throws an {@link EOFException}.
+ *
+ * @return a record that has been transported through the channel or <code>null</code> if currently no record is
+ * available
+ * @throws IOException
+ * thrown if the input channel is already closed {@link EOFException} or a transmission error has occurred
+ */
+// public abstract InputChannelResult readRecord(T target) throws IOException;
+
+ /**
+ * Immediately closes the input channel. The corresponding output channels are
+ * notified if necessary. Any remaining records in any buffers or queue is considered
+ * irrelevant and is discarded.
+ *
+ * @throws InterruptedException
+ * thrown if the thread is interrupted while waiting for the channel to close
+ * @throws IOException
+ * thrown if an I/O error occurs while closing the channel
+ */
+// public abstract void close() throws IOException, InterruptedException;
+
+
+
+ @Override
+ public boolean isInputChannel() {
+ return true;
+ }
+
+
+ @Override
+ public JobID getJobID() {
+ return this.inputGate.getJobID();
+ }
+
+// public abstract AbstractTaskEvent getCurrentEvent();
+
+ public InputChannelResult readRecord(T target) throws IOException {
+ if (this.dataBuffer == null) {
+ if (isClosed()) {
+ return InputChannelResult.END_OF_STREAM;
+ }
+
+ // get the next element we need to handle (buffer or event)
+ BufferOrEvent boe = getNextBufferOrEvent();
+
+ if (boe == null) {
+ throw new IllegalStateException("Input channel was queries for data even though none was announced available.");
+ }
+
+ // handle events
+ if (boe.isEvent())
+ {
+ // sanity check: an event may only come after a complete record.
+ if (this.deserializer.hasUnfinishedData()) {
+ throw new IOException("Channel received an event before completing the current partial record.");
+ }
+
+ AbstractEvent evt = boe.getEvent();
+ if (evt.getClass() == ChannelCloseEvent.class) {
+ this.brokerAggreedToCloseChannel = true;
+ return InputChannelResult.END_OF_STREAM;
+ }
+ else if (evt.getClass() == EndOfSuperstepEvent.class) {
+ return InputChannelResult.END_OF_SUPERSTEP;
+ }
+ else if (evt instanceof AbstractTaskEvent) {
+ this.currentEvent = (AbstractTaskEvent) evt;
+ return InputChannelResult.TASK_EVENT;
+ }
+ else {
+ LOG.error("Received unknown event: " + evt);
+ return InputChannelResult.NONE;
+ }
+ } else {
+ // buffer case
+ this.dataBuffer = boe.getBuffer();
+ this.deserializer.setNextMemorySegment(this.dataBuffer.getMemorySegment(), this.dataBuffer.size());
+ }
+ }
+
+
+ DeserializationResult deserializationResult = this.deserializer.getNextRecord(target);
+
+ if (deserializationResult.isBufferConsumed()) {
+ releasedConsumedReadBuffer(this.dataBuffer);
+ this.dataBuffer = null;
+ }
+
+ if (deserializationResult == DeserializationResult.INTERMEDIATE_RECORD_FROM_BUFFER) {
+ return InputChannelResult.INTERMEDIATE_RECORD_FROM_BUFFER;
+ } else if (deserializationResult == DeserializationResult.LAST_RECORD_FROM_BUFFER) {
+ return InputChannelResult.LAST_RECORD_FROM_BUFFER;
+ } else if (deserializationResult == DeserializationResult.PARTIAL_RECORD) {
+ return InputChannelResult.NONE;
+ } else {
+ throw new IllegalStateException();
+ }
+ }
+
+ @Override
+ public ChannelType getChannelType() {
+ return null;
+ }
+
+ @Override
+ public boolean isClosed() throws IOException{
+ if (this.ioException != null) {
+ throw new IOException("An error occurred in the channel: " + this.ioException.getMessage(), this.ioException);
+ } else {
+ return this.brokerAggreedToCloseChannel;
+ }
+ }
+
+ public void close() throws IOException, InterruptedException {
+
+ this.deserializer.clear();
+ if (this.dataBuffer != null) {
+ releasedConsumedReadBuffer(this.dataBuffer);
+ this.dataBuffer = null;
+ }
+
+ // This code fragment makes sure the isClosed method works in case the channel input has not been fully consumed
+ while (!this.brokerAggreedToCloseChannel)
+ {
+ BufferOrEvent next = getNextBufferOrEvent();
+ if (next != null) {
+ if (next.isEvent()) {
+ if (next.getEvent() instanceof ChannelCloseEvent) {
+ this.brokerAggreedToCloseChannel = true;
+ }
+ } else {
+ releasedConsumedReadBuffer(next.getBuffer());
+ }
+ } else {
+ Thread.sleep(200);
+ }
+ }
+
+ // Send close event to indicate the input channel has successfully
+ // processed all data it is interested in.
+ transferEventToOutputChannel(new ChannelCloseEvent());
+ }
+
+
+ private void releasedConsumedReadBuffer(Buffer buffer) {
+ this.amountOfDataTransmitted += buffer.size();
+ buffer.recycleBuffer();
+ }
+
+
+ public void notifyGateThatInputIsAvailable() {
+ this.getInputGate().notifyRecordIsAvailable(getIndex());
+ }
+
+
+ @Override
+ public void transferEvent(AbstractEvent event) throws IOException, InterruptedException {
+ transferEventToOutputChannel(event);
+ }
+
+
+ public void reportIOException(IOException ioe) {
+ this.ioException = ioe;
+ }
+
+
+ @Override
+ public void releaseAllResources() {
+ this.brokerAggreedToCloseChannel = true;
+ this.deserializer.clear();
+
+ // The buffers are recycled by the input channel wrapper
+ }
+
+ /**
+ * Notify the channel that a data unit has been consumed.
+ */
+ public void notifyDataUnitConsumed() {
+ this.getInputGate().notifyDataUnitConsumed(getIndex());
+ }
+
+ public AbstractTaskEvent getCurrentEvent() {
+ AbstractTaskEvent e = this.currentEvent;
+ this.currentEvent = null;
+ return e;
+ }
+
+ // InputChannelContext
+
+ @Override
+ public void queueEnvelope(Envelope envelope) {
+ // The sequence number of the envelope to be queued
+ final int sequenceNumber = envelope.getSequenceNumber();
+
+ synchronized (this.queuedEnvelopes) {
+
+ if (this.destroyCalled) {
+ final Buffer buffer = envelope.getBuffer();
+ if (buffer != null) {
+ buffer.recycleBuffer();
+ }
+ return;
+ }
+
+ final int expectedSequenceNumber = this.lastReceivedEnvelope + 1;
+ if (sequenceNumber != expectedSequenceNumber) {
+ // This is a problem, now we are actually missing some data
+ reportIOException(new IOException("Expected data packet " + expectedSequenceNumber + " but received " + sequenceNumber));
+
+ // notify that something (an exception) is available
+ notifyGateThatInputIsAvailable();
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Input channel " + this.toString() + " expected envelope " + expectedSequenceNumber
+ + " but received " + sequenceNumber);
+ }
+
+ // rescue the buffer
+ final Buffer buffer = envelope.getBuffer();
+ if (buffer != null) {
+ buffer.recycleBuffer();
+ }
+ } else {
+
+ this.queuedEnvelopes.add(envelope);
+ this.lastReceivedEnvelope = sequenceNumber;
+
+ // Notify the channel about the new data. notify as much as there is (buffer plus once per event)
+ if (envelope.getBuffer() != null) {
+ notifyGateThatInputIsAvailable();
+ }
+
+ List<? extends AbstractEvent> events = envelope.deserializeEvents();
+
+ if (events != null) {
+ for (int i = 0; i < events.size(); i++) {
+ notifyGateThatInputIsAvailable();
+ }
+ }
+ }
+ }
+ }
+
+ @Override
+ public void destroy() {
+ final Queue<Buffer> buffersToRecycle = new ArrayDeque<Buffer>();
+
+ synchronized (this.queuedEnvelopes) {
+ this.destroyCalled = true;
+
+ while (!this.queuedEnvelopes.isEmpty()) {
+ final Envelope envelope = this.queuedEnvelopes.poll();
+ if (envelope.getBuffer() != null) {
+ buffersToRecycle.add(envelope.getBuffer());
+ }
+ }
+ }
+
+ while (!buffersToRecycle.isEmpty()) {
+ buffersToRecycle.poll().recycleBuffer();
+ }
+ }
+
+ public void logQueuedEnvelopes() {
+ int numberOfQueuedEnvelopes = 0;
+ int numberOfQueuedEnvelopesWithMemoryBuffers = 0;
+ int numberOfQueuedEnvelopesWithFileBuffers = 0;
+
+ synchronized (this.queuedEnvelopes) {
+
+ final Iterator<Envelope> it = this.queuedEnvelopes.iterator();
+ while (it.hasNext()) {
+
+ final Envelope envelope = it.next();
+ ++numberOfQueuedEnvelopes;
+ final Buffer buffer = envelope.getBuffer();
+ if (buffer == null) {
+ continue;
+ }
+
+ ++numberOfQueuedEnvelopesWithMemoryBuffers;
+ }
+ }
+
+ System.out.println("\t\t" + this.toString() + ": " + numberOfQueuedEnvelopes + " ("
+ + numberOfQueuedEnvelopesWithMemoryBuffers + ", " + numberOfQueuedEnvelopesWithFileBuffers + ")");
+
+ }
+
+ @Override
+ public Buffer requestBuffer(int minBufferSize) throws IOException {
+ return this.inputGate.requestBuffer(minBufferSize);
+ }
+
+ @Override
+ public Buffer requestBufferBlocking(int minBufferSize) throws IOException, InterruptedException {
+ return this.inputGate.requestBufferBlocking(minBufferSize);
+ }
+
+ @Override
+ public int getBufferSize() {
+ return this.inputGate.getBufferSize();
+ }
+
+ @Override
+ public void reportAsynchronousEvent() {
+ this.inputGate.reportAsynchronousEvent();
+ }
+
+ @Override
+ public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+ return this.inputGate.registerBufferAvailabilityListener(listener);
+ }
+
+ // ChannelBroker
+
+ public BufferOrEvent getNextBufferOrEvent() throws IOException {
+ // return pending events first
+ if (this.pendingEvents != null) {
+ // if the field is not null, it must always have a next value!
+ BufferOrEvent next = new BufferOrEvent(this.pendingEvents.next());
+ if (!this.pendingEvents.hasNext()) {
+ this.pendingEvents = null;
+ }
+ return next;
+ }
+
+ // if no events are pending, get the next buffer
+ Envelope nextEnvelope;
+ synchronized (this.queuedEnvelopes) {
+ if (this.queuedEnvelopes.isEmpty()) {
+ return null;
+ }
+ nextEnvelope = this.queuedEnvelopes.poll();
+ }
+
+ // schedule events as pending, because events come always after the buffer!
+ List<AbstractEvent> events = (List<AbstractEvent>) nextEnvelope.deserializeEvents();
+ Iterator<AbstractEvent> eventsIt = events.iterator();
+ if (eventsIt.hasNext()) {
+ this.pendingEvents = eventsIt;
+ }
+
+ // get the buffer, if there is one
+ if (nextEnvelope.getBuffer() != null) {
+ return new BufferOrEvent(nextEnvelope.getBuffer());
+ }
+ else if (this.pendingEvents != null) {
+ // if the field is not null, it must always have a next value!
+ BufferOrEvent next = new BufferOrEvent(this.pendingEvents.next());
+ if (!this.pendingEvents.hasNext()) {
+ this.pendingEvents = null;
+ }
+
+ return next;
+ }
+ else {
+ // no buffer and no events, this should be an error
+ throw new IOException("Received an envelope with neither data nor events.");
+ }
+ }
+
+ public void transferEventToOutputChannel(AbstractEvent event) throws IOException, InterruptedException {
+ Envelope ephemeralEnvelope = new Envelope(0, getJobID(), getID());
+ ephemeralEnvelope.serializeEventList(Arrays.asList(event));
+
+ this.envelopeDispatcher.dispatchFromInputChannel(ephemeralEnvelope);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/OutputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/OutputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/OutputChannel.java
new file mode 100644
index 0000000..7ca916c
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/OutputChannel.java
@@ -0,0 +1,193 @@
+/***********************************************************************************************************************
+ * 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.channels;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.network.envelope.Envelope;
+import eu.stratosphere.runtime.io.gates.OutputGate;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+
+public class OutputChannel extends Channel {
+
+ private static final Log LOG = LogFactory.getLog(OutputChannel.class);
+
+ private final Object closeLock = new Object();
+
+ private final OutputGate outputGate;
+
+ private boolean senderCloseRequested;
+
+ private boolean receiverCloseRequested;
+
+ private int currentSeqNum;
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ /**
+ * Creates a new output channel object.
+ *
+ * @param outputGate the output gate this channel is connected to
+ * @param index the index of the channel in the output gate
+ * @param id the ID of the channel
+ * @param connectedId the ID of the channel this channel is connected to
+ * @param type the type of this channel
+ */
+ public OutputChannel(OutputGate outputGate, int index, ChannelID id, ChannelID connectedId, ChannelType type) {
+ super(index, id, connectedId, type);
+
+ this.outputGate = outputGate;
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+ // Data processing
+ // -----------------------------------------------------------------------------------------------------------------
+
+ public void sendBuffer(Buffer buffer) throws IOException, InterruptedException {
+ checkStatus();
+
+ // discard empty buffers
+ if (buffer.size() == 0) {
+ return;
+ }
+
+ Envelope envelope = createNextEnvelope();
+ envelope.setBuffer(buffer);
+ this.envelopeDispatcher.dispatchFromOutputChannel(envelope);
+ }
+
+ public void sendEvent(AbstractEvent event) throws IOException, InterruptedException {
+ checkStatus();
+
+ Envelope envelope = createNextEnvelope();
+ envelope.serializeEventList(Arrays.asList(event));
+ this.envelopeDispatcher.dispatchFromOutputChannel(envelope);
+ }
+
+ public void sendBufferAndEvent(Buffer buffer, AbstractEvent event) throws IOException, InterruptedException {
+ checkStatus();
+
+ Envelope envelope = createNextEnvelope();
+ envelope.setBuffer(buffer);
+ envelope.serializeEventList(Arrays.asList(event));
+ this.envelopeDispatcher.dispatchFromOutputChannel(envelope);
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+ // Event processing
+ // -----------------------------------------------------------------------------------------------------------------
+
+ @Override
+ public void queueEnvelope(Envelope envelope) {
+ if (envelope.hasBuffer()) {
+ throw new IllegalStateException("Envelope for OutputChannel has Buffer attached.");
+ }
+
+ for (AbstractEvent event : envelope.deserializeEvents()) {
+ if (event.getClass() == ChannelCloseEvent.class) {
+ synchronized (this.closeLock) {
+ this.receiverCloseRequested = true;
+ this.closeLock.notifyAll();
+ }
+ LOG.debug("OutputChannel received close event from target.");
+ }
+ else if (event instanceof AbstractTaskEvent) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("OutputChannel received task event: " + event);
+ }
+
+ this.outputGate.deliverEvent((AbstractTaskEvent) event);
+ }
+ else {
+ throw new RuntimeException("OutputChannel received an event that is neither close nor task event.");
+ }
+ }
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+ // Shutdown
+ // -----------------------------------------------------------------------------------------------------------------
+
+ public void requestClose() throws IOException, InterruptedException {
+ if (this.senderCloseRequested) {
+ return;
+ }
+
+ this.senderCloseRequested = true;
+
+ Envelope envelope = createNextEnvelope();
+ envelope.serializeEventList(Arrays.asList(new ChannelCloseEvent()));
+ this.envelopeDispatcher.dispatchFromOutputChannel(envelope);
+ }
+
+ @Override
+ public boolean isClosed() {
+ return this.senderCloseRequested && this.receiverCloseRequested;
+ }
+
+ public void waitForChannelToBeClosed() throws InterruptedException {
+ synchronized (this.closeLock) {
+ while (!this.receiverCloseRequested) {
+ this.closeLock.wait(1000);
+ }
+ }
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ @Override
+ public boolean isInputChannel() {
+ return false;
+ }
+
+ @Override
+ public JobID getJobID() {
+ return this.outputGate.getJobID();
+ }
+
+ private void checkStatus() throws IOException {
+ if (this.senderCloseRequested) {
+ throw new IllegalStateException(String.format("Channel %s already requested to be closed.", getID()));
+ }
+ if (this.receiverCloseRequested) {
+ throw new ReceiverAlreadyClosedException();
+ }
+ }
+
+ private Envelope createNextEnvelope() {
+ return new Envelope(this.currentSeqNum++, getJobID(), getID());
+ }
+
+ @Override
+ public void transferEvent(AbstractEvent event) throws IOException, InterruptedException {
+ // TODO remove with pending changes for input side
+ }
+
+ @Override
+ public void releaseAllResources() {
+ // nothing to do for buffer oriented runtime => TODO remove with pending changes for input side
+ }
+
+ @Override
+ public void destroy() {
+ // nothing to do for buffer oriented runtime => TODO remove with pending changes for input side
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ReceiverAlreadyClosedException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ReceiverAlreadyClosedException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ReceiverAlreadyClosedException.java
new file mode 100644
index 0000000..d03179f
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ReceiverAlreadyClosedException.java
@@ -0,0 +1,22 @@
+/***********************************************************************************************************************
+ *
+ * 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.channels;
+
+import java.io.IOException;
+
+
+public class ReceiverAlreadyClosedException extends IOException {
+ private static final long serialVersionUID = 1L;
+}
[26/30] git commit: Add debug log messages to Netty stack
Posted by rm...@apache.org.
Add debug log messages to Netty stack
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/6d8fea96
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/6d8fea96
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/6d8fea96
Branch: refs/heads/master
Commit: 6d8fea9602c7cd248ac29d9d5c2c2f1ade42c73f
Parents: 4cd4a13
Author: uce <u....@fu-berlin.de>
Authored: Mon Jun 2 15:46:07 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Sat Jun 7 09:41:21 2014 +0200
----------------------------------------------------------------------
.../network/netty/InboundEnvelopeDecoder.java | 11 +++++
.../network/netty/NettyConnectionManager.java | 52 ++++++++++++++++++++
.../network/netty/OutboundConnectionQueue.java | 17 ++++++-
.../test/runtime/NetworkStackNepheleITCase.java | 6 +--
4 files changed, 81 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6d8fea96/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
index 1ab1871..54f4617 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
@@ -23,6 +23,8 @@ import eu.stratosphere.runtime.io.network.Envelope;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundHandlerAdapter;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import java.io.IOException;
import java.nio.ByteBuffer;
@@ -30,6 +32,8 @@ import java.util.concurrent.ConcurrentLinkedQueue;
public class InboundEnvelopeDecoder extends ChannelInboundHandlerAdapter implements BufferAvailabilityListener {
+ private static final Log LOG = LogFactory.getLog(InboundEnvelopeDecoder.class);
+
private final BufferProviderBroker bufferProviderBroker;
private final BufferAvailabilityChangedTask bufferAvailabilityChangedTask = new BufferAvailabilityChangedTask();
@@ -123,6 +127,10 @@ public class InboundEnvelopeDecoder extends ChannelInboundHandlerAdapter impleme
case REGISTERED:
if (ctx.channel().config().isAutoRead()) {
ctx.channel().config().setAutoRead(false);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(String.format("Set channel %s auto read to false.", ctx.channel()));
+ }
}
this.stagedBuffer = in;
@@ -194,6 +202,9 @@ public class InboundEnvelopeDecoder extends ChannelInboundHandlerAdapter impleme
if (decodeBuffer(stagedBuffer, channelHandlerContext)) {
stagedBuffer = null;
channelHandlerContext.channel().config().setAutoRead(true);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(String.format("Set channel %s auto read to true.", channelHandlerContext.channel()));
+ }
}
} catch (IOException e) {
availableBuffer.recycleBuffer();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6d8fea96/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManager.java
index ae67f42..4e3f043 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManager.java
@@ -34,6 +34,8 @@ import org.apache.commons.logging.LogFactory;
import java.io.IOException;
import java.net.InetAddress;
+import java.util.Date;
+import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@@ -41,6 +43,8 @@ public class NettyConnectionManager {
private static final Log LOG = LogFactory.getLog(NettyConnectionManager.class);
+ private static final int DEBUG_PRINT_QUEUED_ENVELOPES_EVERY_MS = 10000;
+
private final ChannelManager channelManager;
private final ServerBootstrap in;
@@ -108,6 +112,30 @@ public class NettyConnectionManager {
} catch (InterruptedException e) {
throw new RuntimeException("Could not bind server socket for incoming connections.");
}
+
+ if (LOG.isDebugEnabled()) {
+ new Thread(new Runnable() {
+ @Override
+ public void run() {
+ Date date = new Date();
+
+
+ while (true) {
+ try {
+ Thread.sleep(DEBUG_PRINT_QUEUED_ENVELOPES_EVERY_MS);
+
+ date.setTime(System.currentTimeMillis());
+ System.out.println(date);
+
+ System.out.println(getNonZeroNumQueuedEnvelopes());
+
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }).start();
+ }
}
public void shutdown() {
@@ -170,6 +198,30 @@ public class NettyConnectionManager {
channel.enqueue(envelope);
}
+ private String getNonZeroNumQueuedEnvelopes() {
+ StringBuilder str = new StringBuilder();
+
+ str.append(String.format("==== %d outgoing connections ===\n", this.outConnections.size()));
+
+ for (Map.Entry<RemoteReceiver, Object> entry : this.outConnections.entrySet()) {
+ RemoteReceiver receiver = entry.getKey();
+
+ Object value = entry.getValue();
+ if (value instanceof OutboundConnectionQueue) {
+ OutboundConnectionQueue queue = (OutboundConnectionQueue) value;
+ if (queue.getNumQueuedEnvelopes() > 0) {
+ str.append(String.format("%s> Number of queued envelopes for %s with channel %s: %d\n",
+ Thread.currentThread().getId(), receiver, queue.getChannel(), queue.getNumQueuedEnvelopes()));
+ }
+ } else if (value instanceof ChannelInBuildup) {
+ str.append(String.format("%s> Connection to %s is still in buildup\n",
+ Thread.currentThread().getId(), receiver));
+ }
+ }
+
+ return str.toString();
+ }
+
// ------------------------------------------------------------------------
private static final class ChannelInBuildup implements ChannelFutureListener {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6d8fea96/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundConnectionQueue.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundConnectionQueue.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundConnectionQueue.java
index c687408..b6ec915 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundConnectionQueue.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundConnectionQueue.java
@@ -23,6 +23,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import java.util.ArrayDeque;
+import java.util.concurrent.atomic.AtomicInteger;
public class OutboundConnectionQueue extends ChannelInboundHandlerAdapter implements ChannelFutureListener {
@@ -30,11 +31,12 @@ public class OutboundConnectionQueue extends ChannelInboundHandlerAdapter implem
private final Channel channel;
- private final ArrayDeque<Envelope> queuedEnvelopes;
+ private final ArrayDeque<Envelope> queuedEnvelopes = new ArrayDeque<Envelope>();
+
+ private final AtomicInteger numQueued = new AtomicInteger(0);
public OutboundConnectionQueue(Channel channel) {
this.channel = channel;
- this.queuedEnvelopes = new ArrayDeque<Envelope>();
channel.pipeline().addFirst(this);
}
@@ -56,6 +58,7 @@ public class OutboundConnectionQueue extends ChannelInboundHandlerAdapter implem
boolean triggerWrite = this.queuedEnvelopes.isEmpty();
this.queuedEnvelopes.addLast((Envelope) envelopeToEnqueue);
+ this.numQueued.incrementAndGet();
if (triggerWrite) {
writeAndFlushNextEnvelopeIfPossible();
@@ -80,9 +83,19 @@ public class OutboundConnectionQueue extends ChannelInboundHandlerAdapter implem
}
}
+ public int getNumQueuedEnvelopes() {
+ return this.numQueued.intValue();
+ }
+
+ public Channel getChannel() {
+ return this.channel;
+ }
+
private void writeAndFlushNextEnvelopeIfPossible() {
if (this.channel.isWritable() && !this.queuedEnvelopes.isEmpty()) {
Envelope nextEnvelope = this.queuedEnvelopes.pollFirst();
+ this.numQueued.decrementAndGet();
+
this.channel.writeAndFlush(nextEnvelope).addListener(this);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6d8fea96/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java
index f5beda4..64026a2 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java
@@ -162,8 +162,8 @@ public class NetworkStackNepheleITCase extends RecordAPITestBase {
producer.connectTo(forwarder, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
forwarder.connectTo(consumer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
- producer.setVertexToShareInstancesWith(forwarder);
- forwarder.setVertexToShareInstancesWith(consumer);
+ forwarder.setVertexToShareInstancesWith(producer);
+ consumer.setVertexToShareInstancesWith(producer);
}
else {
producer.connectTo(consumer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
@@ -196,7 +196,7 @@ public class NetworkStackNepheleITCase extends RecordAPITestBase {
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));
+ SpeedTestRecord.RECORD_SIZE, dataMbPerSubtask / 1024.0));
boolean isSlow = getTaskConfiguration().getBoolean(IS_SLOW_SENDER_CONFIG_KEY, false);
[30/30] git commit: Check number of designated buffers before
returning buffer to BufferAvailabilityListener
Posted by rm...@apache.org.
Check number of designated buffers before returning buffer to BufferAvailabilityListener
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/3beca629
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/3beca629
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/3beca629
Branch: refs/heads/master
Commit: 3beca629b8f11073415e8107784b2ffe4368f613
Parents: 6d8fea9
Author: uce <u....@fu-berlin.de>
Authored: Mon Jun 2 16:23:40 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Sat Jun 7 09:41:22 2014 +0200
----------------------------------------------------------------------
.../runtime/io/network/bufferprovider/LocalBufferPool.java | 9 +++++++++
1 file changed, 9 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3beca629/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
index 3eb10c1..6285726 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
@@ -294,6 +294,15 @@ public final class LocalBufferPool implements BufferProvider {
this.globalBufferPool.returnBuffer(buffer);
this.numRequestedBuffers--;
} else {
+ // if the number of designated buffers changed in the meantime, make sure
+ // to return the buffer to the global buffer pool
+ if (this.numRequestedBuffers > this.numDesignatedBuffers) {
+ this.globalBufferPool.returnBuffer(buffer);
+ this.numRequestedBuffers--;
+
+ return;
+ }
+
if (!this.listeners.isEmpty()) {
Buffer availableBuffer = new Buffer(buffer, buffer.size(), this.recycler);
try {
[05/30] Replace custom Java NIO TCP/IP code with Netty 4 library
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManager.java
new file mode 100644
index 0000000..ae67f42
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManager.java
@@ -0,0 +1,251 @@
+/***********************************************************************************************************************
+ * 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."));
+ }
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundConnectionQueue.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundConnectionQueue.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundConnectionQueue.java
new file mode 100644
index 0000000..c687408
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundConnectionQueue.java
@@ -0,0 +1,94 @@
+/***********************************************************************************************************************
+ * 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);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoder.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoder.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoder.java
new file mode 100644
index 0000000..424f2c0
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoder.java
@@ -0,0 +1,65 @@
+/***********************************************************************************************************************
+ * 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();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java
deleted file mode 100644
index f22e6f7..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/***********************************************************************************************************************
- * 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();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java
deleted file mode 100644
index 774ad4e..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java
+++ /dev/null
@@ -1,226 +0,0 @@
-/***********************************************************************************************************************
- * 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);
- }
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java
deleted file mode 100644
index 7df1901..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java
+++ /dev/null
@@ -1,529 +0,0 @@
-/***********************************************************************************************************************
- * 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;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java
deleted file mode 100644
index dde26e3..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java
+++ /dev/null
@@ -1,276 +0,0 @@
-/***********************************************************************************************************************
- * 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());
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java
index f4c8aec..696915f 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java
@@ -15,9 +15,6 @@ package eu.stratosphere.runtime.io.serialization;
import eu.stratosphere.core.io.IOReadableWritable;
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.EOFException;
@@ -62,6 +59,7 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
// check if we can get a full length;
if (nonSpanningRemaining >= 4) {
int len = this.nonSpanningWrapper.readInt();
+
if (len <= nonSpanningRemaining - 4) {
// we can get a full record from here
target.read(this.nonSpanningWrapper);
@@ -156,8 +154,9 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
@Override
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();
+ }
this.segment.get(this.position, b, off, len);
this.position += len;
@@ -230,14 +229,16 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
try {
int b;
while ((b = readUnsignedByte()) != '\n') {
- if (b != '\r')
+ if (b != '\r') {
bld.append((char) b);
+ }
}
}
catch (EOFException eofex) {}
- if (bld.length() == 0)
+ if (bld.length() == 0) {
return null;
+ }
// trim a trailing carriage return
int len = bld.length();
@@ -275,8 +276,9 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
while (count < utflen) {
c = (int) bytearr[count] & 0xff;
- if (c > 127)
+ if (c > 127) {
break;
+ }
count++;
chararr[chararr_count++] = (char) c;
}
@@ -298,21 +300,25 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
case 12:
case 13:
count += 2;
- if (count > utflen)
+ if (count > utflen) {
throw new UTFDataFormatException("malformed input: partial character at end");
+ }
char2 = (int) bytearr[count - 1];
- if ((char2 & 0xC0) != 0x80)
+ if ((char2 & 0xC0) != 0x80) {
throw new UTFDataFormatException("malformed input around byte " + count);
+ }
chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
break;
case 14:
count += 3;
- if (count > utflen)
+ if (count > utflen) {
throw new UTFDataFormatException("malformed input: partial character at end");
+ }
char2 = (int) bytearr[count - 2];
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));
+ }
chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
break;
default:
@@ -325,8 +331,9 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
@Override
public final int skipBytes(int n) throws IOException {
- if (n < 0)
+ if (n < 0) {
throw new IllegalArgumentException();
+ }
int toSkip = Math.min(n, remaining());
this.position += toSkip;
@@ -390,6 +397,7 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
return;
} else {
this.recordLength = this.lengthBuffer.getInt(0);
+
this.lengthBuffer.clear();
segmentPosition = toPut;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java
index e6479fe..a8a53fe 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java
@@ -63,11 +63,13 @@ public class DataInputDeserializer implements DataInput {
}
public void setBuffer(byte[] buffer, int start, int len) {
- if (buffer == null)
+ if (buffer == null) {
throw new NullPointerException();
+ }
- if (start < 0 || len < 0 || start + len >= buffer.length)
+ if (start < 0 || len < 0 || start + len >= buffer.length) {
throw new IllegalArgumentException();
+ }
this.buffer = buffer;
this.position = start;
@@ -144,7 +146,7 @@ public class DataInputDeserializer implements DataInput {
@SuppressWarnings("restriction")
int value = UNSAFE.getInt(this.buffer, BASE_OFFSET + this.position);
if (LITTLE_ENDIAN) {
- value = Integer.reverseBytes(value);
+ value = Integer.reverseBytes(value);
}
this.position += 4;
@@ -183,7 +185,7 @@ public class DataInputDeserializer implements DataInput {
@SuppressWarnings("restriction")
long value = UNSAFE.getLong(this.buffer, BASE_OFFSET + this.position);
if (LITTLE_ENDIAN) {
- value = Long.reverseBytes(value);
+ value = Long.reverseBytes(value);
}
this.position += 8;
return value;
@@ -215,8 +217,9 @@ public class DataInputDeserializer implements DataInput {
while (count < utflen) {
c = (int) bytearr[count] & 0xff;
- if (c > 127)
+ if (c > 127) {
break;
+ }
count++;
chararr[chararr_count++] = (char) c;
}
@@ -240,22 +243,26 @@ public class DataInputDeserializer implements DataInput {
case 13:
/* 110x xxxx 10xx xxxx */
count += 2;
- if (count > utflen)
+ if (count > utflen) {
throw new UTFDataFormatException("malformed input: partial character at end");
+ }
char2 = (int) bytearr[count - 1];
- if ((char2 & 0xC0) != 0x80)
+ if ((char2 & 0xC0) != 0x80) {
throw new UTFDataFormatException("malformed input around byte " + count);
+ }
chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
break;
case 14:
/* 1110 xxxx 10xx xxxx 10xx xxxx */
count += 3;
- if (count > utflen)
+ if (count > utflen) {
throw new UTFDataFormatException("malformed input: partial character at end");
+ }
char2 = (int) bytearr[count - 2];
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));
+ }
chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
break;
default:
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java
index b5171b9..ce088f0 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java
@@ -196,9 +196,9 @@ public class DataOutputSerializer implements DataOutput {
}
}
- if (utflen > 65535)
+ if (utflen > 65535) {
throw new UTFDataFormatException("Encoded string is too long: " + utflen);
-
+ }
else if (this.position > this.buffer.length - utflen - 2) {
resize(utflen + 2);
}
@@ -212,8 +212,9 @@ public class DataOutputSerializer implements DataOutput {
int i = 0;
for (i = 0; i < strlen; i++) {
c = str.charAt(i);
- if (!((c >= 0x0001) && (c <= 0x007F)))
+ if (!((c >= 0x0001) && (c <= 0x007F))) {
break;
+ }
bytearr[count++] = (byte) c;
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java
index 443f8d8..b54496a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java
@@ -68,6 +68,7 @@ public class SpanningRecordSerializer<T extends IOReadableWritable> implements R
// write data and length
record.write(this.serializationBuffer);
+
this.lengthBuffer.putInt(0, this.serializationBuffer.length());
this.dataBuffer = this.serializationBuffer.wrapAsByteBuffer();
@@ -103,8 +104,9 @@ public class SpanningRecordSerializer<T extends IOReadableWritable> implements R
* @param source the {@link ByteBuffer} to copy data from
*/
private void copyToTargetBufferFrom(ByteBuffer source) {
- if (this.targetBuffer == null)
+ if (this.targetBuffer == null) {
return;
+ }
int needed = source.remaining();
int available = this.limit - this.position;
@@ -127,8 +129,9 @@ public class SpanningRecordSerializer<T extends IOReadableWritable> implements R
@Override
public Buffer getCurrentBuffer() {
- if (targetBuffer == null)
+ if (targetBuffer == null) {
return null;
+ }
this.targetBuffer.limitSize(this.position);
return this.targetBuffer;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java
index 09b244f..3b2ad69 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/TestBufferProvider.java
@@ -70,7 +70,7 @@ public class TestBufferProvider implements BufferProvider {
}
@Override
- public boolean registerBufferAvailabilityListener(BufferAvailabilityListener bufferAvailabilityListener) {
+ public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener bufferAvailabilityListener) {
throw new UnsupportedOperationException();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
index c66d821..13fbfbc 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java
@@ -33,7 +33,6 @@ import eu.stratosphere.api.java.typeutils.runtime.record.RecordSerializerFactory
import eu.stratosphere.core.memory.DataInputView;
import eu.stratosphere.core.memory.DataOutputView;
import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.nephele.io.ChannelSelector;
import eu.stratosphere.pact.runtime.plugable.SerializationDelegate;
import eu.stratosphere.pact.runtime.shipping.OutputEmitter;
import eu.stratosphere.pact.runtime.shipping.ShipStrategyType;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
index 0b968d8..2bd5d98 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java
@@ -28,8 +28,6 @@ import org.junit.Test;
import eu.stratosphere.api.common.distributions.DataDistribution;
import eu.stratosphere.api.common.distributions.UniformIntegerDistribution;
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.pact.runtime.shipping.RecordOutputEmitter;
import eu.stratosphere.pact.runtime.shipping.ShipStrategyType;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
index a397312..b715a4e 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
@@ -44,7 +44,9 @@ import eu.stratosphere.util.MutableObjectIterator;
import java.io.IOException;
import java.util.LinkedList;
import java.util.List;
+import java.util.Map;
import java.util.Set;
+import java.util.concurrent.FutureTask;
public class MockEnvironment implements Environment, BufferProvider, LocalBufferPoolOwner {
@@ -124,8 +126,8 @@ public class MockEnvironment implements Environment, BufferProvider, LocalBuffer
}
@Override
- public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
- return false;
+ public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+ return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
}
@Override
@@ -338,4 +340,9 @@ public class MockEnvironment implements Environment, BufferProvider, LocalBuffer
public BufferProvider getOutputBufferProvider() {
return this;
}
+
+ @Override
+ public Map<String, FutureTask<Path>> getCopyTask() {
+ return null;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java
deleted file mode 100644
index b8914a8..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java
+++ /dev/null
@@ -1,394 +0,0 @@
-/***********************************************************************************************************************
- * 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;
- }
- }
-}
[21/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java
deleted file mode 100644
index 173021f..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/LocalChannelWithAccessInfo.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.channels.FileChannel;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-final class LocalChannelWithAccessInfo implements ChannelWithAccessInfo {
-
- /**
- * The logging object.
- */
- private static final Log LOG = LogFactory.getLog(LocalChannelWithAccessInfo.class);
-
- private final File file;
-
- private final FileChannel channel;
-
- private final AtomicLong reservedWritePosition;
-
- private final AtomicInteger referenceCounter;
-
- private final AtomicBoolean deleteOnClose;
-
- LocalChannelWithAccessInfo(final File file, final boolean deleteOnClose) throws IOException {
-
- this.file = file;
- this.channel = new RandomAccessFile(file, "rw").getChannel();
- this.reservedWritePosition = new AtomicLong(0L);
- this.referenceCounter = new AtomicInteger(0);
- this.deleteOnClose = new AtomicBoolean(deleteOnClose);
- }
-
-
- @Override
- public FileChannel getChannel() {
-
- return this.channel;
- }
-
-
- @Override
- public FileChannel getAndIncrementReferences() {
-
- if (incrementReferences()) {
- return this.channel;
- } else {
- return null;
- }
- }
-
- @Override
- public ChannelWithPosition reserveWriteSpaceAndIncrementReferences(final int spaceToReserve) {
-
- if (incrementReferences()) {
- return new ChannelWithPosition(this.channel, this.reservedWritePosition.getAndAdd(spaceToReserve));
- } else {
- return null;
- }
- }
-
-
- @Override
- public int decrementReferences() {
-
- int current = this.referenceCounter.get();
- while (true) {
- if (current <= 0) {
- // this is actually an error case, because the channel was deleted before
- throw new IllegalStateException("The references to the file were already at zero.");
- }
-
- if (current == 1) {
- // this call decrements to zero, so mark it as deleted
- if (this.referenceCounter.compareAndSet(current, Integer.MIN_VALUE)) {
- current = 0;
- break;
- }
- } else if (this.referenceCounter.compareAndSet(current, current - 1)) {
- current = current - 1;
- break;
- }
- current = this.referenceCounter.get();
- }
-
- if (current > 0) {
- return current;
- } else if (current == 0) {
- // delete the channel
- this.referenceCounter.set(Integer.MIN_VALUE);
- this.reservedWritePosition.set(Long.MIN_VALUE);
- try {
- this.channel.close();
- } catch (IOException ioex) {
- if (LOG.isErrorEnabled()) {
- LOG.error("Error while closing spill file for file buffers: " + ioex.getMessage(), ioex);
- }
- }
- if (this.deleteOnClose.get()) {
- this.file.delete();
- }
- return current;
- } else {
- throw new IllegalStateException("The references to the file were already at zero.");
- }
- }
-
-
- @Override
- public boolean incrementReferences() {
-
- int current = this.referenceCounter.get();
- while (true) {
- // check whether it was disposed in the meantime
- if (current < 0) {
- return false;
- }
- // atomically check and increment
- if (this.referenceCounter.compareAndSet(current, current + 1)) {
- return true;
- }
- current = this.referenceCounter.get();
- }
- }
-
-
- @Override
- public void disposeSilently() {
-
- this.referenceCounter.set(Integer.MIN_VALUE);
- this.reservedWritePosition.set(Long.MIN_VALUE);
-
- if (this.channel.isOpen()) {
- try {
- this.channel.close();
- } catch (Throwable t) {
- }
- }
-
- if (this.deleteOnClose.get()) {
- this.file.delete();
- }
- }
-
-
- @Override
- public void updateDeleteOnCloseFlag(final boolean deleteOnClose) {
-
- this.deleteOnClose.compareAndSet(true, deleteOnClose);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java
deleted file mode 100644
index 8b2c0fb..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBuffer.java
+++ /dev/null
@@ -1,249 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.WritableByteChannel;
-
-import eu.stratosphere.core.memory.MemorySegment;
-
-public final class MemoryBuffer extends Buffer {
-
- private final MemoryBufferRecycler bufferRecycler;
-
- private final MemorySegment internalMemorySegment;
-
- /**
- * Internal index that points to the next byte to write
- */
- private int index = 0;
-
- /**
- * Internal limit to simulate ByteBuffer behavior of MemorySegment. index > limit is not allowed.
- */
- private int limit = 0;
-
- MemoryBuffer(final int bufferSize, final MemorySegment memory, final MemoryBufferPoolConnector bufferPoolConnector) {
- if (bufferSize > memory.size()) {
- throw new IllegalArgumentException("Requested segment size is " + bufferSize
- + ", but provided MemorySegment only has a capacity of " + memory.size());
- }
-
- this.bufferRecycler = new MemoryBufferRecycler(memory, bufferPoolConnector);
- this.internalMemorySegment = memory;
- this.position(0);
- this.limit(bufferSize);
- }
-
- private MemoryBuffer(final int bufferSize, final int pos, final MemorySegment memory, final MemoryBufferRecycler bufferRecycler) {
- this.bufferRecycler = bufferRecycler;
- this.internalMemorySegment = memory;
- this.position(pos);
- this.limit(bufferSize);
- }
-
- @Override
- public int read(ByteBuffer dst) throws IOException {
-
- if (!this.hasRemaining()) {
- return -1;
- }
- int numBytes = dst.remaining();
- final int remaining = this.remaining();
- if (numBytes == 0) {
- return 0;
- }
- if(numBytes > remaining) {
- numBytes = remaining;
- }
- internalMemorySegment.get(position(), dst, numBytes);
- index += numBytes;
- return numBytes;
- }
-
-
- @Override
- public int writeTo(WritableByteChannel writableByteChannel) throws IOException {
- if (!this.hasRemaining()) {
- return -1;
- }
-
- final ByteBuffer wrapped = this.internalMemorySegment.wrap(index, limit-index);
- final int written = writableByteChannel.write(wrapped);
- position(wrapped.position());
- return written;
- }
-
-
- @Override
- public void close() throws IOException {
-
- this.position(this.limit());
- }
-
-
- @Override
- public boolean isOpen() {
-
- return this.hasRemaining();
- }
-
- /**
- * Resets the memory buffer.
- *
- * @param bufferSize
- * the size of buffer in bytes after the reset
- */
- public final void reset(final int bufferSize) {
- if(bufferSize > this.internalMemorySegment.size()) {
- throw new RuntimeException("Given buffer size exceeds underlying buffer size");
- }
- this.position(0);
- this.limit(bufferSize);
- }
-
- public final void position(final int i) {
- if(i > limit) {
- throw new IndexOutOfBoundsException("new position is larger than the limit");
- }
- index = i;
- }
-
- @Override
- public final int position() {
- return index;
- }
-
- public final void limit(final int l) {
- if(limit > internalMemorySegment.size()) {
- throw new RuntimeException("Limit is larger than MemoryBuffer size");
- }
- if (index > limit) {
- index = limit;
- }
- limit = l;
- }
-
- public final int limit() {
- return limit;
- }
-
- public final boolean hasRemaining() {
- return index < limit;
- }
-
- public final int remaining() {
- return limit - index;
- }
-
- /**
- * Put MemoryBuffer into read mode
- */
- public final void flip() {
- limit = position();
- position(0);
- }
-
- public void clear() {
- this.limit = getTotalSize();
- this.position(0);
- }
-
- /**
- *
- * @return Returns the size of the underlying MemorySegment
- */
- public int getTotalSize() {
- return this.internalMemorySegment.size();
- }
-
- @Override
- public final int size() {
- return this.limit();
- }
-
- public MemorySegment getMemorySegment() {
- return this.internalMemorySegment;
- }
-
-
- @Override
- protected void recycle() {
- this.bufferRecycler.decreaseReferenceCounter();
- if(bufferRecycler.referenceCounter.get() == 0) {
- clear();
- }
- }
-
-
-
- @Override
- public boolean isBackedByMemory() {
- return true;
- }
-
-
- @Override
- public MemoryBuffer duplicate() {
- final MemoryBuffer duplicatedMemoryBuffer = new MemoryBuffer(this.limit(), this.position(), this.internalMemorySegment, this.bufferRecycler);
- this.bufferRecycler.increaseReferenceCounter();
- return duplicatedMemoryBuffer;
- }
-
-
- @Override
- public void copyToBuffer(final Buffer destinationBuffer) throws IOException {
- if (size() > destinationBuffer.size()) {
- throw new IllegalArgumentException("Destination buffer is too small to store content of source buffer: "
- + size() + " vs. " + destinationBuffer.size());
- }
- final MemoryBuffer target = (MemoryBuffer) destinationBuffer;
- this.internalMemorySegment.copyTo(this.position(), target.getMemorySegment(), destinationBuffer.position(), limit()-position());
- target.position(limit()-position()); // even if we do not change the source (this), we change the destination!!
- destinationBuffer.flip();
- }
-
-
-
- @Override
- public int write(final ByteBuffer src) throws IOException {
- int numBytes = src.remaining();
- final int thisRemaining = this.remaining();
- if(thisRemaining == 0) {
- return 0;
- }
- if(numBytes > thisRemaining) {
- numBytes = thisRemaining;
- }
- this.internalMemorySegment.put(position(), src, numBytes);
- this.index += numBytes;
- return numBytes;
- }
-
-
- @Override
- public int write(final ReadableByteChannel readableByteChannel) throws IOException {
-
- if (!this.hasRemaining()) {
- return 0;
- }
- ByteBuffer wrapper = this.internalMemorySegment.wrap(index, limit-index);
- final int written = readableByteChannel.read(wrapper);
- this.position(wrapper.position());
- this.limit(wrapper.limit());
- return written;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferPoolConnector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferPoolConnector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferPoolConnector.java
deleted file mode 100644
index bd8519d..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferPoolConnector.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import eu.stratosphere.core.memory.MemorySegment;
-
-/**
- * The memory buffer pool connector provides a connection between {@link MemoryBuffer} and the {@link LocalBufferPool}
- * the memory buffer's encapsulated byte buffer has originally been taken from.
- *
- */
-public interface MemoryBufferPoolConnector {
-
- /**
- * Called by the {@link MemoryBufferRecycler} to return a buffer to its original buffer pool.
- *
- * @param byteBuffer
- * the buffer to be recycled
- */
- void recycle(MemorySegment memSeg);
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferRecycler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferRecycler.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferRecycler.java
deleted file mode 100644
index 141e3a1..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/MemoryBufferRecycler.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.memory.MemorySegment;
-
-/**
- * A memory buffer recycler takes care of the correct recycling of the internal byte buffer which backs a memory buffer.
- * Since buffer objects can be duplicated, i.e. multiple buffer objects point to the same physical buffer, it is
- * necessary to coordinate the recycling of the physical buffer.
- * <p>
- * This class is thread-safe.
- *
- */
-public final class MemoryBufferRecycler {
-
- /**
- * The log object used to report debug information and possible errors.
- */
- private static final Log LOG = LogFactory.getLog(MemoryBufferRecycler.class);
-
- /**
- * The original memory segment which has been taken from byte buffered channel manager's buffer pool.
- */
- private final MemorySegment originalSegment;
-
- /**
- * The connection to the pool from which the byte buffer has originally been taken.
- */
- private final MemoryBufferPoolConnector bufferPoolConnector;
-
- /**
- * The number of memory buffer objects which may still access the physical buffer.
- */
- public final AtomicInteger referenceCounter = new AtomicInteger(1);
-
- /**
- * Constructs a new memory buffer recycler.
- *
- * @param originalBuffer
- * the original byte buffer
- * @param bufferPoolConnector
- * the connection to the pool from which the byte buffer has originally been taken
- */
- MemoryBufferRecycler(final MemorySegment originalSegment, final MemoryBufferPoolConnector bufferPoolConnector) {
-
- this.originalSegment = originalSegment;
- this.bufferPoolConnector = bufferPoolConnector;
- }
-
- /**
- * Increases the number of references to the physical buffer by one.
- */
- void increaseReferenceCounter() {
-
- if (this.referenceCounter.getAndIncrement() == 0) {
- LOG.error("Increasing reference counter from 0 to 1");
- }
- }
-
- /**
- * Decreases the number of references to the physical buffer by one. If the number of references becomes zero the
- * physical buffer is recycled.
- */
- void decreaseReferenceCounter() {
-
- final int val = this.referenceCounter.decrementAndGet();
- if (val == 0) {
- this.bufferPoolConnector.recycle(this.originalSegment);
-
- } else if (val < 0) {
- LOG.error("reference counter is negative");
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/SerializationBuffer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/SerializationBuffer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/SerializationBuffer.java
deleted file mode 100644
index 55bcb56..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/SerializationBuffer.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.WritableByteChannel;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.DataOutputBuffer;
-
-/**
- * A class for serializing a record to its binary representation.
- *
- * @param <T>
- * the type of the record this serialization buffer can be used for
- */
-public class SerializationBuffer<T extends IOReadableWritable> {
-
- private static final int SIZEOFINT = 4;
-
- private DataOutputBuffer serializationBuffer = new DataOutputBuffer();
-
- private ByteBuffer lengthBuf = ByteBuffer.allocate(SIZEOFINT);
-
- private int bytesReadFromBuffer = 0;
-
- /**
- * Translates an integer into an array of bytes.
- *
- * @param val
- * The integer to be translated
- * @param arr
- * The byte buffer to store the data of the integer
- */
- private void integerToByteBuffer(final int val, final ByteBuffer byteBuffer) {
-
- for (int i = 0; i < SIZEOFINT; ++i) {
- final int shift = i << (SIZEOFINT - 1); // i * 8
- byteBuffer.put(SIZEOFINT - 1 - i, (byte) ((val & (0xff << shift)) >>> shift));
- }
-
- byteBuffer.position(0);
- byteBuffer.limit(SIZEOFINT);
- }
-
- /**
- * Return <code>true</code> if the internal serialization buffer still contains data.
- * In this case the method serialize must not be called. If the internal buffer is empty
- * the method return <code>false</code>
- *
- * @return <code>true</code> if the internal serialization buffer still contains data, <code>false</code> it it is
- * empty
- */
- public boolean dataLeftFromPreviousSerialization() {
- return leftInSerializationBuffer() > 0;
- }
-
- /**
- * Reads the internal serialization buffer and writes the data to the given {@link WritableByteChannel} byte
- * channel.
- *
- * @param writableByteChannel
- * the byte channel to write the serialized data to
- * @return the number of bytes written the to given byte channel
- * @throws IOException
- * thrown if an error occurs while writing to serialized data to the channel
- */
- public int read(final WritableByteChannel writableByteChannel) throws IOException {
-
- int bytesReadFromLengthBuf = 0;
-
- // Deal with length buffer first
- if (this.lengthBuf.hasRemaining()) { // There is data from the length buffer to be written
- bytesReadFromLengthBuf = writableByteChannel.write(this.lengthBuf);
- }
-
- final int bytesReadFromSerializationBuf = writableByteChannel.write(this.serializationBuffer.getData());
- // byteBuffer.put(this.serializationBuffer.getData(), this.bytesReadFromBuffer, length);
- this.bytesReadFromBuffer += bytesReadFromSerializationBuf;
-
- if (leftInSerializationBuffer() == 0) { // Record is entirely written to byteBuffer
- this.serializationBuffer.reset();
- this.bytesReadFromBuffer = 0;
- }
-
- return (bytesReadFromSerializationBuf + bytesReadFromLengthBuf);
- }
-
- /**
- * Return the number of bytes that have not been read from the internal serialization
- * buffer so far.
- *
- * @return the number of bytes that have not been read from the internal serialization buffer so far
- */
- private int leftInSerializationBuffer() {
-
- return (this.serializationBuffer.getLength() - this.bytesReadFromBuffer);
- }
-
- /**
- * Serializes the record and writes it to an internal buffer. The buffer grows dynamically
- * in case more memory is required to serialization.
- *
- * @param record
- * The record to the serialized
- * @throws IOException
- * Thrown if data from a previous serialization process is still in the internal buffer and has not yet been
- * transfered to a byte buffer
- */
- public void serialize(final T record) throws IOException {
-
- // Check if there is data left in the buffer
- if (dataLeftFromPreviousSerialization()) {
- throw new IOException("Cannot write new data, " + leftInSerializationBuffer()
- + " bytes still left from previous call");
- }
-
- record.write(this.serializationBuffer); // serializationBuffer grows dynamically
-
- // Now record is completely in serializationBuffer;
- integerToByteBuffer(this.serializationBuffer.getLength(), this.lengthBuf);
- }
-
- public void clear() {
- this.bytesReadFromBuffer = 0;
- this.lengthBuf.clear();
- this.serializationBuffer.reset();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java
deleted file mode 100644
index 0009d5a..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedInputChannel.java
+++ /dev/null
@@ -1,243 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.bytebuffered;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.io.InputChannelResult;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.RecordDeserializer;
-import eu.stratosphere.nephele.io.channels.AbstractInputChannel;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-
-/**
- * @param <T> The type of record that can be transported through this channel.
- */
-public abstract class AbstractByteBufferedInputChannel<T extends IOReadableWritable> extends AbstractInputChannel<T> {
-
- /**
- * The log object used to report warnings and errors.
- */
- private static final Log LOG = LogFactory.getLog(AbstractByteBufferedInputChannel.class);
-
- /**
- * The deserializer used to deserialize records.
- */
- private final RecordDeserializer<T> deserializer;
-
- /**
- * Buffer for the uncompressed (raw) data.
- */
- private Buffer dataBuffer;
-
- private ByteBufferedInputChannelBroker inputChannelBroker;
-
- private AbstractTaskEvent currentEvent;
-
- /**
- * The exception observed in this channel while processing the buffers. Checked and thrown
- * per-buffer.
- */
- private volatile IOException ioException;
-
- /**
- * Stores the number of bytes read through this input channel since its instantiation.
- */
- private long amountOfDataTransmitted;
-
-
- private volatile boolean brokerAggreedToCloseChannel;
-
- /**
- * Creates a new input channel.
- *
- * @param inputGate
- * the input gate this channel is wired to
- * @param channelIndex
- * the channel's index at the associated input gate
- * @param type
- * the type of record transported through this channel
- * @param channelID
- * the ID of the channel
- * @param connectedChannelID
- * the ID of the channel this channel is connected to
- */
- public AbstractByteBufferedInputChannel(final InputGate<T> inputGate, final int channelIndex,
- final RecordDeserializer<T> deserializer, final ChannelID channelID, final ChannelID connectedChannelID) {
- super(inputGate, channelIndex, channelID, connectedChannelID);
- this.deserializer = deserializer;
- }
-
- @Override
- public InputChannelResult readRecord(T target) throws IOException {
- if (this.dataBuffer == null) {
- if (isClosed()) {
- return InputChannelResult.END_OF_STREAM;
- }
-
- // get the next element we need to handle (buffer or event)
- BufferOrEvent boe = this.inputChannelBroker.getNextBufferOrEvent();
-
- if (boe == null) {
- throw new IllegalStateException("Input channel was queries for data even though none was announced available.");
- }
-
- // handle events
- if (boe.isEvent())
- {
- // sanity check: an event may only come after a complete record.
- if (this.deserializer.hasUnfinishedData()) {
- throw new IOException("Channel received an event before completing the current partial record.");
- }
-
- AbstractEvent evt = boe.getEvent();
- if (evt.getClass() == ByteBufferedChannelCloseEvent.class) {
- this.brokerAggreedToCloseChannel = true;
- return InputChannelResult.END_OF_STREAM;
- }
- else if (evt.getClass() == EndOfSuperstepEvent.class) {
- return InputChannelResult.END_OF_SUPERSTEP;
- }
- else if (evt instanceof AbstractTaskEvent) {
- this.currentEvent = (AbstractTaskEvent) evt;
- return InputChannelResult.TASK_EVENT;
- }
- else {
- LOG.error("Received unknown event: " + evt);
- return InputChannelResult.NONE;
- }
- } else {
- // buffer case
- this.dataBuffer = boe.getBuffer();
- }
- }
-
- // get the next record form the buffer
- T nextRecord = this.deserializer.readData(target, this.dataBuffer);
-
- // release the buffer if it is empty
- if (this.dataBuffer.remaining() == 0) {
- releasedConsumedReadBuffer(this.dataBuffer);
- this.dataBuffer = null;
- return nextRecord == null ? InputChannelResult.NONE : InputChannelResult.LAST_RECORD_FROM_BUFFER;
- } else {
- return nextRecord == null ? InputChannelResult.NONE : InputChannelResult.INTERMEDIATE_RECORD_FROM_BUFFER;
- }
- }
-
- @Override
- public boolean isClosed() throws IOException{
- if (this.ioException != null) {
- throw new IOException("An error occurred in the channel: " + this.ioException.getMessage(), this.ioException);
- } else {
- return this.brokerAggreedToCloseChannel;
- }
- }
-
-
- @Override
- public void close() throws IOException, InterruptedException {
-
- this.deserializer.clear();
- if (this.dataBuffer != null) {
- releasedConsumedReadBuffer(this.dataBuffer);
- this.dataBuffer = null;
- }
-
- // This code fragment makes sure the isClosed method works in case the channel input has not been fully consumed
- while (!this.brokerAggreedToCloseChannel)
- {
- BufferOrEvent next = this.inputChannelBroker.getNextBufferOrEvent();
- if (next != null) {
- if (next.isEvent()) {
- if (next.getEvent() instanceof ByteBufferedChannelCloseEvent) {
- this.brokerAggreedToCloseChannel = true;
- }
- } else {
- releasedConsumedReadBuffer(next.getBuffer());
- }
- } else {
- Thread.sleep(200);
- }
- }
-
- // Send close event to indicate the input channel has successfully
- // processed all data it is interested in.
- transferEvent(new ByteBufferedChannelCloseEvent());
- }
-
-
- private void releasedConsumedReadBuffer(Buffer buffer) {
- this.amountOfDataTransmitted += buffer.size();
- buffer.recycleBuffer();
- }
-
-
- public void setInputChannelBroker(ByteBufferedInputChannelBroker inputChannelBroker) {
- this.inputChannelBroker = inputChannelBroker;
- }
-
-
- public void notifyGateThatInputIsAvailable() {
- this.getInputGate().notifyRecordIsAvailable(getChannelIndex());
- }
-
-
- @Override
- public void transferEvent(AbstractEvent event) throws IOException, InterruptedException {
- this.inputChannelBroker.transferEventToOutputChannel(event);
- }
-
-
- public void reportIOException(IOException ioe) {
- this.ioException = ioe;
- }
-
-
- @Override
- public void releaseAllResources() {
- this.brokerAggreedToCloseChannel = true;
- this.deserializer.clear();
-
- // The buffers are recycled by the input channel wrapper
- }
-
-
- @Override
- public long getAmountOfDataTransmitted() {
- return this.amountOfDataTransmitted;
- }
-
-
- /**
- * Notify the channel that a data unit has been consumed.
- */
- public void notifyDataUnitConsumed() {
- this.getInputGate().notifyDataUnitConsumed(getChannelIndex());
- }
-
- @Override
- public AbstractTaskEvent getCurrentEvent() {
- AbstractTaskEvent e = this.currentEvent;
- this.currentEvent = null;
- return e;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java
deleted file mode 100644
index 50d4a50..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/AbstractByteBufferedOutputChannel.java
+++ /dev/null
@@ -1,255 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.bytebuffered;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.channels.AbstractOutputChannel;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.channels.SerializationBuffer;
-
-public abstract class AbstractByteBufferedOutputChannel<T extends IOReadableWritable> extends AbstractOutputChannel<T> {
-
- /**
- * The serialization buffer used to serialize records.
- */
- private final SerializationBuffer<T> serializationBuffer = new SerializationBuffer<T>();
-
- /**
- * Buffer for the serialized output data.
- */
- private Buffer dataBuffer = null;
-
- /**
- * Stores whether the channel is requested to be closed.
- */
- private boolean closeRequested = false;
-
- /**
- * The output channel broker the channel should contact to request and release write buffers.
- */
- private ByteBufferedOutputChannelBroker outputChannelBroker = null;
-
-
- /**
- * Stores the number of bytes transmitted through this output channel since its instantiation.
- */
- private long amountOfDataTransmitted = 0L;
-
- private static final Log LOG = LogFactory.getLog(AbstractByteBufferedOutputChannel.class);
-
- /**
- * Creates a new byte buffered output channel.
- *
- * @param outputGate
- * the output gate this channel is wired to
- * @param channelIndex
- * the channel's index at the associated output gate
- * @param channelID
- * the ID of the channel
- * @param connectedChannelID
- * the ID of the channel this channel is connected to
- */
- protected AbstractByteBufferedOutputChannel(final OutputGate<T> outputGate, final int channelIndex,
- final ChannelID channelID, final ChannelID connectedChannelID) {
- super(outputGate, channelIndex, channelID, connectedChannelID);
- }
-
-
- @Override
- public boolean isClosed() throws IOException, InterruptedException {
-
- if (this.closeRequested && this.dataBuffer == null
- && !this.serializationBuffer.dataLeftFromPreviousSerialization()) {
-
- if (!this.outputChannelBroker.hasDataLeftToTransmit()) {
- return true;
- }
- }
-
- return false;
- }
-
-
- @Override
- public void requestClose() throws IOException, InterruptedException {
-
- if (!this.closeRequested) {
- this.closeRequested = true;
- if (this.serializationBuffer.dataLeftFromPreviousSerialization()) {
- // make sure we serialized all data before we send the close event
- flush();
- }
-
- if (getType() == ChannelType.INMEMORY || !isBroadcastChannel() || getChannelIndex() == 0) {
- transferEvent(new ByteBufferedChannelCloseEvent());
- flush();
- }
- }
- }
-
- /**
- * Requests a new write buffer from the framework. This method blocks until the requested buffer is available.
- *
- * @throws InterruptedException
- * thrown if the thread is interrupted while waiting for the buffer
- * @throws IOException
- * thrown if an I/O error occurs while waiting for the buffer
- */
- private void requestWriteBufferFromBroker() throws InterruptedException, IOException {
- if (Thread.interrupted()) {
- throw new InterruptedException();
- }
- this.dataBuffer = this.outputChannelBroker.requestEmptyWriteBuffer();
- }
-
- /**
- * Returns the filled buffer to the framework and triggers further processing.
- *
- * @throws IOException
- * thrown if an I/O error occurs while releasing the buffers
- * @throws InterruptedException
- * thrown if the thread is interrupted while releasing the buffers
- */
- private void releaseWriteBuffer() throws IOException, InterruptedException {
- // Keep track of number of bytes transmitted through this channel
- this.amountOfDataTransmitted += this.dataBuffer.size();
-
- this.outputChannelBroker.releaseWriteBuffer(this.dataBuffer);
- this.dataBuffer = null;
- }
-
-
- @Override
- public void writeRecord(T record) throws IOException, InterruptedException {
-
- // Get a write buffer from the broker
- if (this.dataBuffer == null) {
- requestWriteBufferFromBroker();
- }
-
- if (this.closeRequested) {
- throw new IOException("Channel is aready requested to be closed");
- }
-
- // Check if we can accept new records or if there are still old
- // records to be transmitted
- while (this.serializationBuffer.dataLeftFromPreviousSerialization()) {
-
- this.serializationBuffer.read(this.dataBuffer);
- if (this.dataBuffer.remaining() == 0) {
- releaseWriteBuffer();
- requestWriteBufferFromBroker();
- }
- }
-
- if (this.serializationBuffer.dataLeftFromPreviousSerialization()) {
- throw new IOException("Serialization buffer is expected to be empty!");
- }
-
- this.serializationBuffer.serialize(record);
-
- this.serializationBuffer.read(this.dataBuffer);
- if (this.dataBuffer.remaining() == 0) {
- releaseWriteBuffer();
- }
- }
-
- /**
- * Sets the output channel broker this channel should contact to request and release write buffers.
- *
- * @param byteBufferedOutputChannelBroker
- * the output channel broker the channel should contact to request and release write buffers
- */
- public void setByteBufferedOutputChannelBroker(ByteBufferedOutputChannelBroker byteBufferedOutputChannelBroker) {
-
- this.outputChannelBroker = byteBufferedOutputChannelBroker;
- }
-
-
- public void processEvent(AbstractEvent event) {
-
- if (event instanceof AbstractTaskEvent) {
- getOutputGate().deliverEvent((AbstractTaskEvent) event);
- } else {
- LOG.error("Channel " + getID() + " received unknown event " + event);
- }
- }
-
-
- @Override
- public void transferEvent(AbstractEvent event) throws IOException, InterruptedException {
-
- flush();
- this.outputChannelBroker.transferEventToInputChannel(event);
- }
-
-
- @Override
- public void flush() throws IOException, InterruptedException {
-
- // Get rid of remaining data in the serialization buffer
- while (this.serializationBuffer.dataLeftFromPreviousSerialization()) {
-
- if (this.dataBuffer == null) {
-
- try {
- requestWriteBufferFromBroker();
- } catch (InterruptedException e) {
- LOG.error(e);
- }
- }
- this.serializationBuffer.read(this.dataBuffer);
- if (this.dataBuffer.remaining() == 0) {
- releaseWriteBuffer();
- }
- }
-
- // Get rid of the leased write buffer
- if (this.dataBuffer != null) {
- releaseWriteBuffer();
- }
- }
-
-
- @Override
- public void releaseAllResources() {
-
- // TODO: Reconsider release of broker's resources here
- this.closeRequested = true;
-
- this.serializationBuffer.clear();
-
- if (this.dataBuffer != null) {
- this.dataBuffer.recycleBuffer();
- this.dataBuffer = null;
- }
- }
-
-
- @Override
- public long getAmountOfDataTransmitted() {
-
- return this.amountOfDataTransmitted;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/BufferOrEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/BufferOrEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/BufferOrEvent.java
deleted file mode 100644
index f89c7fb..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/BufferOrEvent.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.bytebuffered;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.io.channels.Buffer;
-
-/**
- * Either type for {@link Buffer} and {@link AbstractEvent}.
- */
-public class BufferOrEvent {
-
- private final Buffer buffer;
-
- private final AbstractEvent event;
-
- public BufferOrEvent(Buffer buffer) {
- this.buffer = buffer;
- this.event = null;
- }
-
- public BufferOrEvent(AbstractEvent event) {
- this.buffer = null;
- this.event = event;
- }
-
- public boolean isBuffer() {
- return this.buffer != null;
- }
-
- public boolean isEvent() {
- return this.event != null;
- }
-
- public Buffer getBuffer() {
- return this.buffer;
- }
-
- public AbstractEvent getEvent() {
- return this.event;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedChannelCloseEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedChannelCloseEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedChannelCloseEvent.java
deleted file mode 100644
index f427b70..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedChannelCloseEvent.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.bytebuffered;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-
-public class ByteBufferedChannelCloseEvent extends AbstractEvent {
-
- @Override
- public void read(DataInput in) throws IOException {
-
- // Nothing to do here
- }
-
- @Override
- public void write(DataOutput out) throws IOException {
- // Nothing to do here
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedInputChannelBroker.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedInputChannelBroker.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedInputChannelBroker.java
deleted file mode 100644
index 4d259ec..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedInputChannelBroker.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.bytebuffered;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-
-
-public interface ByteBufferedInputChannelBroker {
-
- public BufferOrEvent getNextBufferOrEvent() throws IOException;
-
- /**
- * Forwards the given event to the connected network output channel on a best effort basis.
- *
- * @param event
- * the event to be transferred
- * @throws InterruptedException
- * thrown if the thread is interrupted while waiting for the event to be transfered
- * @throws IOException
- * thrown if an I/O error occurs while transferring the event
- */
- void transferEventToOutputChannel(AbstractEvent event) throws IOException, InterruptedException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.java
deleted file mode 100644
index af45e4c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/ByteBufferedOutputChannelBroker.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.bytebuffered;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.io.channels.Buffer;
-
-public interface ByteBufferedOutputChannelBroker {
-
- /**
- * Requests an empty write buffer from the broker. This method will block
- * until the requested write buffer is available.
- *
- * @return the byte buffers to write in
- * @throws InterruptedException
- * thrown if the connected task is interrupted while waiting for the buffer
- * @throws IOException
- * thrown if an error occurs while requesting the empty write buffer.
- */
- Buffer requestEmptyWriteBuffer() throws InterruptedException, IOException;
-
- /**
- * Returns a filled write buffer to the broker. The broker will take care
- * of the buffers and transfer the user data to the connected input channel on a best effort basis.
- *
- * @param buffer
- * the buffer to be returned to the broker
- * @throws InterruptedException
- * thrown if the thread is interrupted while waiting for the buffers to be released
- * @throws IOException
- * thrown if an I/O error occurs while releasing the buffers
- */
- void releaseWriteBuffer(Buffer buffer) throws IOException, InterruptedException;
-
- /**
- * Checks if there is still data created by this output channel that must be transfered to the corresponding input
- * channel.
- *
- * @return <code>true</code> if the channel has data left to transmit, <code>false</code> otherwise
- * @throws InterruptedException
- * thrown if the connected task is interrupted while waiting for the remaining data to be transmitted
- * @throws IOException
- * thrown if an error occurs while transmitting the remaining data
- */
- boolean hasDataLeftToTransmit() throws IOException, InterruptedException;
-
- /**
- * Forwards the given event to the connected network input channel on a best effort basis.
- *
- * @param event
- * the event to be transferred
- * @throws InterruptedException
- * thrown if the thread is interrupted while waiting for the event to be transfered
- * @throws IOException
- * thrown if an I/O error occurs while transfering the event
- */
- void transferEventToInputChannel(AbstractEvent event) throws IOException, InterruptedException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/EndOfSuperstepEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/EndOfSuperstepEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/EndOfSuperstepEvent.java
deleted file mode 100644
index b0b20d4..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/EndOfSuperstepEvent.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.bytebuffered;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-
-/**
- * Marks the end of a superstep of one particular iteration head
- */
-public class EndOfSuperstepEvent extends AbstractEvent {
-
- public static final EndOfSuperstepEvent INSTANCE = new EndOfSuperstepEvent();
-
- @Override
- public void write(DataOutput out) throws IOException {}
-
- @Override
- public void read(DataInput in) throws IOException {}
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryInputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryInputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryInputChannel.java
deleted file mode 100644
index 0609fcb..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryInputChannel.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.bytebuffered;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.RecordDeserializer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-
-public final class InMemoryInputChannel<T extends IOReadableWritable> extends AbstractByteBufferedInputChannel<T> {
-
- public InMemoryInputChannel(InputGate<T> inputGate, int channelIndex, RecordDeserializer<T> deserializer,
- ChannelID channelID, ChannelID connectedChannelID) {
- super(inputGate, channelIndex, deserializer, channelID, connectedChannelID);
- }
-
- @Override
- public ChannelType getType() {
-
- return ChannelType.INMEMORY;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryOutputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryOutputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryOutputChannel.java
deleted file mode 100644
index 0113d6f..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/InMemoryOutputChannel.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.bytebuffered;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-
-public final class InMemoryOutputChannel<T extends IOReadableWritable> extends AbstractByteBufferedOutputChannel<T> {
-
- public InMemoryOutputChannel(OutputGate<T> outputGate, int channelIndex, ChannelID channelID,
- ChannelID connectedChannelID) {
- super(outputGate, channelIndex, channelID, connectedChannelID);
- }
-
- @Override
- public ChannelType getType() {
-
- return ChannelType.INMEMORY;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkInputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkInputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkInputChannel.java
deleted file mode 100644
index ee11c0e..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkInputChannel.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.bytebuffered;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.RecordDeserializer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-
-public final class NetworkInputChannel<T extends IOReadableWritable> extends AbstractByteBufferedInputChannel<T> {
-
- public NetworkInputChannel(InputGate<T> inputGate, int channelIndex, RecordDeserializer<T> deserializer,
- ChannelID channelID, ChannelID connectedChannelID) {
- super(inputGate, channelIndex, deserializer, channelID, connectedChannelID);
- }
-
- @Override
- public ChannelType getType() {
-
- return ChannelType.NETWORK;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkOutputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkOutputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkOutputChannel.java
deleted file mode 100644
index c98f41a..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/bytebuffered/NetworkOutputChannel.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.bytebuffered;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-
-public final class NetworkOutputChannel<T extends IOReadableWritable> extends AbstractByteBufferedOutputChannel<T> {
-
- public NetworkOutputChannel(OutputGate<T> outputGate, int channelIndex, ChannelID channelID,
- ChannelID connectedChannelID) {
- super(outputGate, channelIndex, channelID, connectedChannelID);
- }
-
- @Override
- public ChannelType getType() {
-
- return ChannelType.NETWORK;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryReader.java
deleted file mode 100644
index acfddde..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryReader.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.library;
-
-import java.util.Iterator;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.fs.FSDataInputStream;
-import eu.stratosphere.core.fs.FileInputSplit;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractFileInputTask;
-import eu.stratosphere.nephele.types.FileRecord;
-
-public class DirectoryReader extends AbstractFileInputTask {
-
- /**
- * The record writer to write the output strings to.
- */
- private RecordWriter<FileRecord> output = null;
-
- // buffer
- private byte[] buffer;
-
- private static final Log LOG = LogFactory.getLog(DirectoryReader.class);
-
-
- @Override
- public void invoke() throws Exception {
-
- final Iterator<FileInputSplit> splitIterator = getFileInputSplits();
- FileRecord fr = null;
-
- while (splitIterator.hasNext()) {
-
- final FileInputSplit split = splitIterator.next();
-
- final long start = split.getStart();
- final long end = start + split.getLength();
-
- if (buffer == null || buffer.length < end - start) {
- buffer = new byte[(int) (end - start)];
- }
-
- if (fr == null || fr.getFileName().compareTo(split.getPath().getName()) != 0) {
- if (fr != null) {
- try {
- output.emit(fr);
- } catch (InterruptedException e) {
- // TODO: Respond to interruption properly
- LOG.error(e);
- }
- }
- fr = new FileRecord(split.getPath().getName());
- }
-
- final FileSystem fs = FileSystem.get(split.getPath().toUri());
-
- final FSDataInputStream fdis = fs.open(split.getPath());
- fdis.seek(split.getStart());
-
- int read = fdis.read(buffer, 0, buffer.length);
- if (read == -1) {
- continue;
- }
-
- fr.append(buffer, 0, read);
-
- if (read != end - start) {
- System.err.println("Unexpected number of bytes read! Expected: " + (end - start) + " Read: " + read);
- }
- }
-
- if (fr != null) {
- try {
- output.emit(fr);
- } catch (InterruptedException e) {
- // TODO: Respond to interruption properly
- LOG.error(e);
- }
- }
- }
-
-
- @Override
- public void registerInputOutput() {
- output = new RecordWriter<FileRecord>(this, FileRecord.class);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryWriter.java
deleted file mode 100644
index 915aaa7..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/DirectoryWriter.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.library;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.fs.FSDataOutputStream;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.template.AbstractFileOutputTask;
-import eu.stratosphere.nephele.types.FileRecord;
-
-public class DirectoryWriter extends AbstractFileOutputTask {
-
- /**
- * The record reader to read the incoming strings from.
- */
- private RecordReader<FileRecord> input = null;
-
- private static final Log LOG = LogFactory.getLog(DirectoryWriter.class);
-
- @Override
- public void invoke() throws Exception {
-
- final Path path = getFileOutputPath();
- final FileSystem fs = path.getFileSystem();
-
- try {
- while (input.hasNext()) {
-
- final FileRecord record = input.next();
- Path newPath = new Path(path + Path.SEPARATOR + record.getFileName());
- FSDataOutputStream outputStream = fs.create(newPath, true);
-
- outputStream.write(record.getDataBuffer(), 0, record.getDataBuffer().length);
- outputStream.close();
- // TODO: Implement me
- // System.out.println(input.next());
- // ODO Auto-generated catch block
- }
- } catch (InterruptedException e) {
- // TODO Auto-generated catch block
- LOG.error(e);
- }
-
- }
-
- @Override
- public void registerInputOutput() {
- input = new RecordReader<FileRecord>(this, FileRecord.class);
-
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/FileLineReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/FileLineReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/FileLineReader.java
deleted file mode 100644
index a26c56c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/FileLineReader.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.library;
-
-import java.util.Iterator;
-
-import eu.stratosphere.core.fs.FSDataInputStream;
-import eu.stratosphere.core.fs.FileInputSplit;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractFileInputTask;
-import eu.stratosphere.runtime.fs.LineReader;
-
-/**
- * A file line reader reads the associated file input splits line by line and outputs the lines as string records.
- *
- */
-public class FileLineReader extends AbstractFileInputTask {
-
- private RecordWriter<StringRecord> output = null;
-
- @Override
- public void invoke() throws Exception {
-
- final Iterator<FileInputSplit> splitIterator = getFileInputSplits();
-
- while (splitIterator.hasNext()) {
-
- final FileInputSplit split = splitIterator.next();
-
- long start = split.getStart();
- long length = split.getLength();
-
- final FileSystem fs = FileSystem.get(split.getPath().toUri());
-
- final FSDataInputStream fdis = fs.open(split.getPath());
-
- final LineReader lineReader = new LineReader(fdis, start, length, (1024 * 1024));
-
- byte[] line = lineReader.readLine();
-
- while (line != null) {
-
- // Create a string object from the data read
- StringRecord str = new StringRecord();
- str.set(line);
-
- // Send out string
- output.emit(str);
-
- line = lineReader.readLine();
- }
-
- // Close the stream;
- lineReader.close();
- }
- }
-
- @Override
- public void registerInputOutput() {
- output = new RecordWriter<StringRecord>(this, StringRecord.class);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/FileLineWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/FileLineWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/FileLineWriter.java
deleted file mode 100644
index 5be3e1b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/library/FileLineWriter.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.library;
-
-import eu.stratosphere.core.fs.FSDataOutputStream;
-import eu.stratosphere.core.fs.FileStatus;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.template.AbstractFileOutputTask;
-
-/**
- * A file line writer reads string records its input gate and writes them to the associated output file.
- *
- */
-public class FileLineWriter extends AbstractFileOutputTask {
-
- /**
- * The record reader through which incoming string records are received.
- */
- private RecordReader<StringRecord> input = null;
-
-
- @Override
- public void invoke() throws Exception {
-
- Path outputPath = getFileOutputPath();
-
- FileSystem fs = FileSystem.get(outputPath.toUri());
- if (fs.exists(outputPath)) {
- FileStatus status = fs.getFileStatus(outputPath);
-
- if (status.isDir()) {
- outputPath = new Path(outputPath.toUri().toString() + "/file_" + getIndexInSubtaskGroup() + ".txt");
- }
- }
-
- final FSDataOutputStream outputStream = fs.create(outputPath, true);
-
- while (this.input.hasNext()) {
-
- StringRecord record = this.input.next();
- byte[] recordByte = (record.toString() + "\r\n").getBytes();
- outputStream.write(recordByte, 0, recordByte.length);
- }
-
- outputStream.close();
-
- }
-
-
- @Override
- public void registerInputOutput() {
- this.input = new RecordReader<StringRecord>(this, StringRecord.class);
- }
-
-
- @Override
- public int getMaximumNumberOfSubtasks() {
- // The default implementation always returns -1
- return -1;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Client.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Client.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Client.java
index 0204ec6..ea7bd4a 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Client.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Client.java
@@ -19,6 +19,16 @@
package eu.stratosphere.nephele.ipc;
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.nephele.net.NetUtils;
+import eu.stratosphere.nephele.util.IOUtils;
+import eu.stratosphere.runtime.io.serialization.DataOutputSerializer;
+import eu.stratosphere.util.ClassUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import javax.net.SocketFactory;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.DataInputStream;
@@ -31,24 +41,13 @@ import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.SocketTimeoutException;
import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
import java.util.Hashtable;
import java.util.Iterator;
import java.util.Map.Entry;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
-import javax.net.SocketFactory;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.DataOutputBuffer;
-import eu.stratosphere.nephele.net.NetUtils;
-import eu.stratosphere.nephele.util.IOUtils;
-import eu.stratosphere.util.ClassUtils;
-
/**
* A client for an IPC service. IPC calls take a single {@link Writable} as a
* parameter, and return a {@link Writable} as their value. A service runs on
@@ -379,13 +378,14 @@ public class Client {
out.write(Server.HEADER.array());
// Write out the ConnectionHeader
- DataOutputBuffer buf = new DataOutputBuffer();
+ DataOutputSerializer buf = new DataOutputSerializer(4 + header.getProtocol().getBytes().length + 1);
header.write(buf);
// Write out the payload length
- int bufLen = buf.getLength();
+ ByteBuffer wrapper = buf.wrapAsByteBuffer();
+ int bufLen = wrapper.limit();
out.writeInt(bufLen);
- out.write(buf.getData().array(), 0, bufLen);
+ out.write(wrapper.array(), 0, bufLen);
}
/*
@@ -456,30 +456,27 @@ public class Client {
return;
}
- DataOutputBuffer d = null;
+ DataOutputSerializer d = null;
try {
synchronized (this.out) {
// for serializing the
// data to be written
- d = new DataOutputBuffer();
+ d = new DataOutputSerializer(64);
// First, write call id to buffer d
d.writeInt(call.id);
// Then write RPC data (the actual call) to buffer d
call.param.write(d);
- byte[] data = d.getData().array();
- int dataLength = d.getLength();
+ ByteBuffer wrapper = d.wrapAsByteBuffer();
+ byte[] data = wrapper.array();
+ int dataLength = wrapper.limit();
out.writeInt(dataLength); // first put the data length
out.write(data, 0, dataLength);// write the data
out.flush();
}
} catch (IOException e) {
markClosed(e);
- } finally {
- // the buffer is just an in-memory buffer, but it is still polite to
- // close early
- IOUtils.closeStream(d);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
index 6af9d74..b4c51f2 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java
@@ -23,8 +23,7 @@ import eu.stratosphere.configuration.IllegalConfigurationException;
import eu.stratosphere.core.io.IOReadableWritable;
import eu.stratosphere.core.io.StringRecord;
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.nephele.template.AbstractInvokable;
import eu.stratosphere.nephele.util.EnumUtils;
import eu.stratosphere.util.StringUtils;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/DistributionPattern.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/DistributionPattern.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/DistributionPattern.java
new file mode 100644
index 0000000..3b8c9a0
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/DistributionPattern.java
@@ -0,0 +1,33 @@
+/***********************************************************************************************************************
+ * 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.jobgraph;
+
+/**
+ * A distribution pattern determines which subtasks of a producing Nephele task a wired to which
+ * subtasks of a consuming subtask.
+ *
+ */
+
+public enum DistributionPattern {
+
+ /**
+ * Each subtask of the producing Nephele task is wired to each subtask of the consuming Nephele task.
+ */
+ BIPARTITE,
+
+ /**
+ * The i-th subtask of the producing Nephele task is wired to the i-th subtask of the consuming Nephele task.
+ */
+ POINTWISE
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobEdge.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobEdge.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobEdge.java
index 51396ff..45788dd 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobEdge.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobEdge.java
@@ -13,8 +13,7 @@
package eu.stratosphere.nephele.jobgraph;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
/**
* Objects of this class represent edges in the user's job graph.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
index e4467a0..2ec2ed6 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobID.java
@@ -1,5 +1,5 @@
/***********************************************************************************************************************
- * 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
* the License. You may obtain a copy of the License at
@@ -13,81 +13,44 @@
package eu.stratosphere.nephele.jobgraph;
+import java.nio.ByteBuffer;
+
import javax.xml.bind.DatatypeConverter;
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
-/**
- * A class for statistically unique job IDs.
- * <p>
- * This class is thread-safe.
- *
- */
public final class JobID extends AbstractID {
- /**
- * Constructs a new random ID from a uniform distribution.
- */
public JobID() {
super();
}
- /**
- * Constructs a new job ID.
- *
- * @param lowerPart
- * the lower bytes of the ID
- * @param upperPart
- * the higher bytes of the ID
- */
- private JobID(final long lowerPart, final long upperPart) {
+ public JobID(long lowerPart, long upperPart) {
super(lowerPart, upperPart);
}
- /**
- * Constructs a new job ID from the given bytes.
- *
- * @param bytes
- * the bytes to initialize the job ID with
- */
- public JobID(final byte[] bytes) {
+ public JobID(byte[] bytes) {
super(bytes);
}
- /**
- * Generates a new statistically unique job ID.
- *
- * @return a new statistically unique job ID
- */
public static JobID generate() {
-
- final long lowerPart = AbstractID.generateRandomBytes();
- final long upperPart = AbstractID.generateRandomBytes();
+ long lowerPart = AbstractID.generateRandomLong();
+ long upperPart = AbstractID.generateRandomLong();
return new JobID(lowerPart, upperPart);
}
- /**
- * Constructs a new job ID and initializes it with the given bytes.
- *
- * @param bytes
- * the bytes to initialize the new job ID with
- * @return the new job ID
- */
- public static JobID fromByteArray(final byte[] bytes) {
-
+ public static JobID fromByteArray(byte[] bytes) {
return new JobID(bytes);
}
-
- /**
- * Constructs a new job ID and initializes it with the given bytes.
- *
- * @param bytes
- * the bytes to initialize the new job ID with
- * @return the new job ID
- */
- public static JobID fromHexString(final String hexString) {
+ public static JobID fromByteBuffer(ByteBuffer buf, int offset) {
+ long lower = buf.getLong(offset);
+ long upper = buf.getLong(offset + 8);
+ return new JobID(lower, upper);
+ }
+
+ public static JobID fromHexString(String hexString) {
return new JobID(DatatypeConverter.parseHexBinary(hexString));
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobVertexID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobVertexID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobVertexID.java
index c6f63ca..689cc02 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobVertexID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobVertexID.java
@@ -13,7 +13,7 @@
package eu.stratosphere.nephele.jobgraph;
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
/**
* A class for statistically unique job vertex IDs.
[20/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
index eba81a2..846ca2e 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/JobManager.java
@@ -75,7 +75,7 @@ import eu.stratosphere.nephele.instance.InstanceManager;
import eu.stratosphere.nephele.instance.InstanceType;
import eu.stratosphere.nephele.instance.InstanceTypeDescription;
import eu.stratosphere.nephele.instance.local.LocalInstanceManager;
-import eu.stratosphere.nephele.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.nephele.ipc.RPC;
import eu.stratosphere.nephele.ipc.Server;
import eu.stratosphere.nephele.jobgraph.AbstractJobVertex;
@@ -90,8 +90,6 @@ import eu.stratosphere.nephele.jobmanager.splitassigner.InputSplitManager;
import eu.stratosphere.nephele.jobmanager.splitassigner.InputSplitWrapper;
import eu.stratosphere.nephele.jobmanager.web.WebInfoServer;
import eu.stratosphere.nephele.managementgraph.ManagementGraph;
-import eu.stratosphere.nephele.managementgraph.ManagementVertexID;
-import eu.stratosphere.nephele.multicast.MulticastManager;
import eu.stratosphere.nephele.profiling.JobManagerProfiler;
import eu.stratosphere.nephele.profiling.ProfilingUtils;
import eu.stratosphere.nephele.protocols.AccumulatorProtocol;
@@ -103,11 +101,10 @@ import eu.stratosphere.nephele.services.accumulators.AccumulatorEvent;
import eu.stratosphere.nephele.taskmanager.AbstractTaskResult;
import eu.stratosphere.nephele.taskmanager.TaskCancelResult;
import eu.stratosphere.nephele.taskmanager.TaskExecutionState;
-import eu.stratosphere.nephele.taskmanager.TaskKillResult;
import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ConnectionInfoLookupResponse;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.RemoteReceiver;
-import eu.stratosphere.nephele.taskmanager.runtime.ExecutorThreadFactory;
+import eu.stratosphere.runtime.io.network.ConnectionInfoLookupResponse;
+import eu.stratosphere.runtime.io.network.RemoteReceiver;
+import eu.stratosphere.nephele.taskmanager.ExecutorThreadFactory;
import eu.stratosphere.nephele.topology.NetworkTopology;
import eu.stratosphere.nephele.types.IntegerRecord;
import eu.stratosphere.nephele.util.SerializableArrayList;
@@ -141,8 +138,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
private final InputSplitManager inputSplitManager;
private final AbstractScheduler scheduler;
-
- private final MulticastManager multicastManager;
private AccumulatorManager accumulatorManager;
@@ -246,9 +241,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
throw new Exception("Unable to load scheduler " + schedulerClassName);
}
- // Create multicastManager
- this.multicastManager = new MulticastManager(this.scheduler);
-
// Load profiler if it should be used
if (GlobalConfiguration.getBoolean(ProfilingUtils.ENABLE_PROFILING_KEY, false)) {
final String profilerClassName = GlobalConfiguration.getString(ProfilingUtils.JOBMANAGER_CLASSNAME_KEY,
@@ -732,8 +724,7 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
@Override
- public ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectionInfo caller, final JobID jobID,
- final ChannelID sourceChannelID) {
+ public ConnectionInfoLookupResponse lookupConnectionInfo(InstanceConnectionInfo caller, JobID jobID, ChannelID sourceChannelID) {
final ExecutionGraph eg = this.scheduler.getExecutionGraphByID(jobID);
if (eg == null) {
@@ -754,7 +745,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
if (sourceChannelID.equals(edge.getInputChannelID())) {
// Request was sent from an input channel
-
final ExecutionVertex connectedVertex = edge.getOutputGate().getVertex();
final AbstractInstance assignedInstance = connectedVertex.getAllocatedResource().getInstance();
@@ -768,9 +758,11 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
// Check execution state
final ExecutionState executionState = connectedVertex.getExecutionState();
if (executionState == ExecutionState.FINISHED) {
- return ConnectionInfoLookupResponse.createReceiverFoundAndReady();
+ // that should not happen. if there is data pending, the receiver cannot be ready
+ return ConnectionInfoLookupResponse.createReceiverNotFound();
}
+ // running is common, finishing is happens when the lookup is for the close event
if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.FINISHING) {
// LOG.info("Created receiverNotReady for " + connectedVertex + " in state " + executionState + " 2");
return ConnectionInfoLookupResponse.createReceiverNotReady();
@@ -781,74 +773,53 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
return ConnectionInfoLookupResponse.createReceiverFoundAndReady(edge.getOutputChannelID());
} else {
// Receiver runs on a different task manager
-
final InstanceConnectionInfo ici = assignedInstance.getInstanceConnectionInfo();
- final InetSocketAddress isa = new InetSocketAddress(ici.getAddress(), ici.getDataPort());
+ final InetSocketAddress isa = new InetSocketAddress(ici.address(), ici.dataPort());
- return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, edge
- .getConnectionID()));
+ return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, edge.getConnectionID()));
}
}
+ // else, the request is for an output channel
+ // Find vertex of connected input channel
+ final ExecutionVertex targetVertex = edge.getInputGate().getVertex();
- if (edge.isBroadcast()) {
+ // Check execution state
+ final ExecutionState executionState = targetVertex.getExecutionState();
- return multicastManager.lookupConnectionInfo(caller, jobID, sourceChannelID);
+ // check whether the task needs to be deployed
+ if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.FINISHING && executionState != ExecutionState.FINISHED) {
- } else {
-
- // Find vertex of connected input channel
- final ExecutionVertex targetVertex = edge.getInputGate().getVertex();
-
- // Check execution state
- final ExecutionState executionState = targetVertex.getExecutionState();
-
- if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.FINISHING
- && executionState != ExecutionState.FINISHED) {
-
- if (executionState == ExecutionState.ASSIGNED) {
-
- final Runnable command = new Runnable() {
-
- /**
- * {@inheritDoc}
- */
- @Override
- public void run() {
- scheduler.deployAssignedVertices(targetVertex);
- }
- };
-
- eg.executeCommand(command);
- }
-
- // LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 3");
- return ConnectionInfoLookupResponse.createReceiverNotReady();
- }
-
- final AbstractInstance assignedInstance = targetVertex.getAllocatedResource().getInstance();
- if (assignedInstance == null) {
- LOG.error("Cannot resolve lookup: vertex found for channel ID " + edge.getInputChannelID()
- + " but no instance assigned");
- // LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 4");
- return ConnectionInfoLookupResponse.createReceiverNotReady();
+ if (executionState == ExecutionState.ASSIGNED) {
+ final Runnable command = new Runnable() {
+ @Override
+ public void run() {
+ scheduler.deployAssignedVertices(targetVertex);
+ }
+ };
+ eg.executeCommand(command);
}
- if (assignedInstance.getInstanceConnectionInfo().equals(caller)) {
- // Receiver runs on the same task manager
- return ConnectionInfoLookupResponse.createReceiverFoundAndReady(edge.getInputChannelID());
- } else {
- // Receiver runs on a different task manager
- final InstanceConnectionInfo ici = assignedInstance.getInstanceConnectionInfo();
- final InetSocketAddress isa = new InetSocketAddress(ici.getAddress(), ici.getDataPort());
+ // LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 3");
+ return ConnectionInfoLookupResponse.createReceiverNotReady();
+ }
- return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, edge
- .getConnectionID()));
- }
+ final AbstractInstance assignedInstance = targetVertex.getAllocatedResource().getInstance();
+ if (assignedInstance == null) {
+ LOG.error("Cannot resolve lookup: vertex found for channel ID " + edge.getInputChannelID() + " but no instance assigned");
+ // LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 4");
+ return ConnectionInfoLookupResponse.createReceiverNotReady();
}
- // LOG.error("Receiver(s) not found");
+ if (assignedInstance.getInstanceConnectionInfo().equals(caller)) {
+ // Receiver runs on the same task manager
+ return ConnectionInfoLookupResponse.createReceiverFoundAndReady(edge.getInputChannelID());
+ } else {
+ // Receiver runs on a different task manager
+ final InstanceConnectionInfo ici = assignedInstance.getInstanceConnectionInfo();
+ final InetSocketAddress isa = new InetSocketAddress(ici.address(), ici.dataPort());
- // return ConnectionInfoLookupResponse.createReceiverNotFound();
+ return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, edge.getConnectionID()));
+ }
}
/**
@@ -921,40 +892,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
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
public void killInstance(final StringRecord instanceName) throws IOException {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java
index c9e02d3..24e2970 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/AbstractScheduler.java
@@ -210,7 +210,7 @@ public abstract class AbstractScheduler implements InstanceListener {
case NETWORK:
deployTarget = false;
break;
- case INMEMORY:
+ case IN_MEMORY:
deployTarget = true;
break;
default:
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java
index efbaf93..762b494 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/scheduler/RecoveryLogic.java
@@ -32,8 +32,7 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
import eu.stratosphere.nephele.instance.AbstractInstance;
import eu.stratosphere.nephele.instance.DummyInstance;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.taskmanager.AbstractTaskResult.ReturnCode;
+import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.nephele.taskmanager.TaskCancelResult;
import eu.stratosphere.nephele.util.SerializableHashSet;
import eu.stratosphere.util.StringUtils;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java
index 8ecd99f..58486da 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdge.java
@@ -13,7 +13,7 @@
package eu.stratosphere.nephele.managementgraph;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
/**
* This class implements a directed edge of a {@link ManagementGraph}. The edge is derived from a channel of the actual
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java
index 20107a7..d263999 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementEdgeID.java
@@ -13,8 +13,8 @@
package eu.stratosphere.nephele.managementgraph;
-import eu.stratosphere.nephele.io.AbstractID;
-import eu.stratosphere.nephele.io.channels.ChannelID;
+import eu.stratosphere.nephele.AbstractID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
/**
* A management edge ID uniquely identifies a {@link ManagementEdge}.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java
index 73548be..63aa6f7 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGateID.java
@@ -13,7 +13,7 @@
package eu.stratosphere.nephele.managementgraph;
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
/**
* A management gate ID uniquely identifies a {@link ManagementGate}.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java
index f4a9855..374656b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java
@@ -28,7 +28,7 @@ import java.util.Map;
import eu.stratosphere.core.io.IOReadableWritable;
import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.nephele.util.EnumUtils;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupEdge.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupEdge.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupEdge.java
index 01a0903..7e6d554 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupEdge.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupEdge.java
@@ -13,7 +13,7 @@
package eu.stratosphere.nephele.managementgraph;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
/**
* This class implements a directed edge of between two {@link ManagementGroupVertex} objects. The edge is derived from
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java
index bbf5fd2..b98a153 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java
@@ -24,7 +24,7 @@ import java.util.Map;
import eu.stratosphere.core.io.IOReadableWritable;
import eu.stratosphere.nephele.execution.ExecutionState;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.nephele.util.EnumUtils;
import eu.stratosphere.util.StringUtils;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertexID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertexID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertexID.java
index b5c1055..aef64af 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertexID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertexID.java
@@ -15,7 +15,7 @@ package eu.stratosphere.nephele.managementgraph;
import javax.xml.bind.DatatypeConverter;
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
/**
* A management group vertex ID uniquely identifies a {@link ManagementGroupVertex}.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertexID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertexID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertexID.java
index 716ae7f..486b3fa 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertexID.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertexID.java
@@ -13,7 +13,7 @@
package eu.stratosphere.nephele.managementgraph;
-import eu.stratosphere.nephele.io.AbstractID;
+import eu.stratosphere.nephele.AbstractID;
/**
* A management vertex ID uniquely identifies a {@link ManagementVertex}.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastCluster.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastCluster.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastCluster.java
deleted file mode 100644
index 87e0181..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastCluster.java
+++ /dev/null
@@ -1,210 +0,0 @@
-/***********************************************************************************************************************
- * 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.multicast;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-
-/**
- * This class represents a cluster of hosts within a multicast-tree.
- *
- */
-
-public class MulticastCluster {
- TreeNode master = null;
-
- HashSet<TreeNode> clusternodes = new HashSet<TreeNode>();
-
- public void addNode(TreeNode node) {
- this.clusternodes.add(node);
- }
-
- public int getSize() {
- return this.clusternodes.size();
- }
-
- public HashSet<TreeNode> getNodes() {
- return this.clusternodes;
- }
-
- /**
- * Returns the master-node of the current cluster.
- *
- * @return
- */
- public TreeNode getMaster() {
-
- if (this.master == null) {
-
- // TODO: topology-aware!!
- if (clusternodes.size() != 0) {
- this.master = clusternodes.iterator().next();
- } else {
- System.out.println("cluster is empty.");
- return null;
- }
-
- }
-
- return this.master;
- }
-
- /**
- * Splits the cluster into an arbitrary number of clusters not exceeding maxsize.
- *
- * @param maxsize
- * @return
- */
- public HashSet<MulticastCluster> splitCluster(int maxsize) {
- // TODO: topology-aware!
- HashSet<MulticastCluster> newClusters = new HashSet<MulticastCluster>();
-
- MulticastCluster actualcluster = new MulticastCluster();
-
- for (Iterator<TreeNode> i = this.clusternodes.iterator(); i.hasNext();) {
- if (actualcluster.getSize() < maxsize) {
- actualcluster.addNode(i.next());
- } else {
- // cluster is full.. add old cluster to list
- newClusters.add(actualcluster);
- // and create new cluster object
- actualcluster = new MulticastCluster();
- actualcluster.addNode(i.next());
- }
- }
-
- newClusters.add(actualcluster);
-
- return newClusters;
-
- }
-
- public static MulticastCluster createInitialCluster(Collection<TreeNode> nodes) {
- // TODO: topology-aware? in here?
-
- MulticastCluster cluster = new MulticastCluster();
-
- for (TreeNode n : nodes) {
- cluster.addNode(n);
- }
-
- return cluster;
-
- }
-
- public static MulticastForwardingTable createClusteredTree(LinkedList<TreeNode> nodes, int maxclustersize) {
-
- return null;
- /*
- // List to store all levels of the clustered multicast tree
- LinkedList<HashSet<MulticastCluster>> clusterlist = new LinkedList<HashSet<MulticastCluster>>();
-
- // Poll off the sending node first..
- TreeNode source = nodes.pollFirst();
-
- // Create an initital multicast cluster containing all receivers
- MulticastCluster initialcluster = createInitialCluster(nodes);
-
- // Create a first layer of clusters with arbitrary size by splitting the initital cluster
- HashSet<MulticastCluster> firstlaycluster = initialcluster.splitCluster(maxclustersize);
-
- // add to the list of cluster layers
- clusterlist.add(firstlaycluster);
-
- // we want the top layer to consist of max. maxclustersize clusters
- while (clusterlist.getFirst().size() > maxclustersize) {
-
- // creating a new cluster-layer...
- MulticastCluster nextlayercluster = new MulticastCluster();
-
- HashSet<MulticastCluster> lowerlayer = clusterlist.getFirst();
-
- // add all master nodes from current layer to next-layer cluster...
- for (MulticastCluster c : lowerlayer) {
- nextlayercluster.addNode(c.getMaster());
- }
-
- // if our next-layer cluster is still too big, we need to split it again
- HashSet<MulticastCluster> nextlayerclusters = nextlayercluster.splitCluster(maxclustersize);
-
- // and finally ad the new layer of clusters
- clusterlist.addFirst(nextlayerclusters);
-
- }
-
- // now we can create the tree...
-
- MulticastForwardingTable table = new MulticastForwardingTable();
-
- HashSet<MulticastCluster> initialclusterlevel = clusterlist.getFirst();
-
- ConnectionInfoLookupResponse sourceentry = ConnectionInfoLookupResponse.createReceiverFoundAndReady();
-
- // add all local targets
- for (ChannelID id : source.getLocalTargets()) {
- System.out.println("local target: " + id);
- sourceentry.addLocalTarget(id);
- }
-
- // connect source node with all master nodes in top-level clusters
- for (MulticastCluster c : initialclusterlevel) {
- sourceentry.addRemoteTarget(c.getMaster().getConnectionInfo());
- }
-
- table.addConnectionInfo(source.getConnectionInfo(), sourceentry);
- System.out.println("forwards for node: " + source.getConnectionInfo());
- System.out.println(sourceentry);
- // now we have connected the source node to the initial cluster layer. iterate through cluster layers and
- // connect
-
- while (clusterlist.size() > 0) {
- HashSet<MulticastCluster> actualclusterlevel = clusterlist.pollFirst();
-
- // add remote targets!
-
- for (MulticastCluster c : actualclusterlevel) {
- TreeNode master = c.getMaster();
- for (Iterator<TreeNode> i = c.getNodes().iterator(); i.hasNext();) {
- TreeNode actualnode = i.next();
- if (!actualnode.equals(master)) {
- // add remote target at master of current cluster
- master.addRemoteTarget(actualnode.getConnectionInfo());
- }
- }
- }
-
- }
-
- // now iterate through all nodes and create forwarding table...
- // we already have the entry for the source node..
- for (TreeNode n : nodes) {
- ConnectionInfoLookupResponse actualentry = ConnectionInfoLookupResponse.createReceiverFoundAndReady();
- for (ChannelID localTarget : n.getLocalTargets()) {
- actualentry.addLocalTarget(localTarget);
- }
- for (InstanceConnectionInfo remoteTarget : n.getRemoteTargets()) {
- actualentry.addRemoteTarget(remoteTarget);
- }
- table.addConnectionInfo(n.getConnectionInfo(), actualentry);
- System.out.println("forwards for node: " + n.getConnectionInfo());
- System.out.println(actualentry);
- }
-
- return table;
- */
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastForwardingTable.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastForwardingTable.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastForwardingTable.java
deleted file mode 100644
index d08821e..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastForwardingTable.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/***********************************************************************************************************************
- * 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.multicast;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ConnectionInfoLookupResponse;
-
-/**
- * This class contains ConnectionInfoLookupResponse objects containing local, as well as remote receivers for all
- * instances within a certain job-specific multicast tree.
- *
- */
-public class MulticastForwardingTable {
-
- private final Map<InstanceConnectionInfo, ConnectionInfoLookupResponse> forwardingTable = new HashMap<InstanceConnectionInfo, ConnectionInfoLookupResponse>();
-
- /**
- * Returns the related ConnectionInfoLookupResponse for the calling Instance.
- *
- * @param caller
- * @return
- */
- public ConnectionInfoLookupResponse getConnectionInfo(InstanceConnectionInfo caller) {
- if (this.forwardingTable.containsKey(caller)) {
- return this.forwardingTable.get(caller);
- } else {
- return null;
- }
- }
-
- protected void addConnectionInfo(InstanceConnectionInfo caller, ConnectionInfoLookupResponse response) {
- this.forwardingTable.put(caller, response);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastManager.java
deleted file mode 100644
index 0a7b471..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/MulticastManager.java
+++ /dev/null
@@ -1,463 +0,0 @@
-/***********************************************************************************************************************
- * 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.multicast;
-
-import java.io.BufferedReader;
-import java.io.DataInputStream;
-import java.io.FileInputStream;
-import java.io.InputStreamReader;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.nephele.execution.ExecutionState;
-import eu.stratosphere.nephele.executiongraph.ExecutionEdge;
-import eu.stratosphere.nephele.executiongraph.ExecutionGate;
-import eu.stratosphere.nephele.executiongraph.ExecutionGraph;
-import eu.stratosphere.nephele.executiongraph.ExecutionVertex;
-import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.jobmanager.JobManager;
-import eu.stratosphere.nephele.jobmanager.scheduler.AbstractScheduler;
-import eu.stratosphere.nephele.protocols.ChannelLookupProtocol;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ConnectionInfoLookupResponse;
-
-/**
- * The MulticastManager is responsible for the creation and storage of application-layer multicast trees used to
- * broadcast records to multiple target vertices.
- *
- */
-
-public final class MulticastManager implements ChannelLookupProtocol {
-
- /**
- * The log object used to report errors and warnings.
- */
- private static final Log LOG = LogFactory.getLog(JobManager.class);
-
- /**
- * Indicates if the arrangement of nodes within the overlay-tree should be randomized or not. If set to false,
- * arrangement of the same set of receiver nodes is guaranteed to be the same
- */
- private final boolean randomized;
-
- /**
- * Indicates if the tree should be constructed with a given topology stored in a file.
- */
- private final boolean useHardCodedTree;
-
- /**
- * File containing the hard-coded tree topology, if desired should contain node names (e.g. hostnames) with
- * corresponding children per line.
- * For example, a line "vm1.local vm2.local vm3.local" would result in vm1.local connecting to vm2.local and
- * vm3.local as children no further checking for connectivity of the given topology is done!
- */
- private final String hardCodedTreeFilePath;
-
- /**
- * Indicates the desired branching of the generated multicast-tree. 0 means unicast transmisison, 1 sequential tree,
- * 2 binomial tree, 3+ clustered tree
- */
- private final int treeBranching;
-
- /**
- * Reference to the scheduler.
- */
- private final AbstractScheduler scheduler;
-
- /**
- * Map caching already computed multicast forwarding tables.
- */
- private final Map<ChannelID, MulticastForwardingTable> cachedTrees = new HashMap<ChannelID, MulticastForwardingTable>();
-
- /**
- * Constructs a new multicast manager.
- *
- * @param scheduler
- * reference to the scheduler
- */
- public MulticastManager(final AbstractScheduler scheduler) {
-
- this.scheduler = scheduler;
-
- this.randomized = GlobalConfiguration.getBoolean("multicast.randomize", false);
- this.treeBranching = GlobalConfiguration.getInteger("multicast.branching", 1);
- this.useHardCodedTree = GlobalConfiguration.getBoolean("multicast.usehardcodedtree", false);
- this.hardCodedTreeFilePath = GlobalConfiguration.getString("multicast.hardcodedtreefile", null);
- }
-
- /**
- * Retrieves all recipients of a data for the given <code>sourceChannelID</code>. Returns both local recipients as
- * well as next-hop remote instances within the multicast-tree.
- *
- * @param caller
- * the {@link InstanceConnectionInfo} object of the task manager which calls this method
- * @param jobID
- * the ID of the job the channel ID belongs to
- * @param sourceChannelID
- * the ID of the channel to resolve
- * @return the lookup response containing the connection info and a return code
- */
- public synchronized ConnectionInfoLookupResponse lookupConnectionInfo(final InstanceConnectionInfo caller,
- final JobID jobID, final ChannelID sourceChannelID) {
-
- if (LOG.isInfoEnabled()) {
- LOG.info("Receiving multicast receiver request from " + caller + " channel ID: " + sourceChannelID);
- }
-
- // Check if the tree is already created and cached
- if (this.cachedTrees.containsKey(sourceChannelID)) {
-
- LOG.info("Replying with cached entry...");
- return cachedTrees.get(sourceChannelID).getConnectionInfo(caller);
-
- } else {
-
- // No tree exists, so we assume that this is the sending node initiating a multicast
-
- if (!checkIfAllTargetVerticesReady(caller, jobID, sourceChannelID)) {
- LOG.info("Received multicast request but not all receivers ready.");
-
- return ConnectionInfoLookupResponse.createReceiverNotReady();
- }
-
- // Receivers are up and running.. extract tree nodes...
- LinkedList<TreeNode> treeNodes = extractTreeNodes(caller, jobID, sourceChannelID, this.randomized);
-
- // Do we want to use a hard-coded tree topology?
- if (this.useHardCodedTree) {
- LOG.info("Creating a hard-coded tree topology from file: " + hardCodedTreeFilePath);
- cachedTrees.put(sourceChannelID, createHardCodedTree(treeNodes));
- return cachedTrees.get(sourceChannelID).getConnectionInfo(caller);
- }
-
- // Otherwise we create a default tree and put it into the tree-cache
- cachedTrees.put(sourceChannelID, createDefaultTree(treeNodes, this.treeBranching));
- return cachedTrees.get(sourceChannelID).getConnectionInfo(caller);
-
- }
-
- }
-
- /**
- * Returns and removes the TreeNode which is closest to the given indicator.
- *
- * @param indicator
- * @param nodes
- * @return
- */
- private TreeNode pollClosestNode(final TreeNode indicator, final LinkedList<TreeNode> nodes) {
-
- TreeNode closestnode = getClosestNode(indicator, nodes);
-
- nodes.remove(closestnode);
-
- return closestnode;
-
- }
-
- /**
- * Returns the TreeNode which is closest to the given indicator Node. Proximity is determined
- * either using topology-information (if given), penalty information (if given) or it returns
- * the first node in the list.
- *
- * @param indicator
- * @param nodes
- * @return
- */
- private TreeNode getClosestNode(final TreeNode indicator, final LinkedList<TreeNode> nodes) {
-
- if (indicator == null) {
- return nodes.getFirst();
- }
-
- TreeNode closestNode = null;
- for (TreeNode n : nodes) {
- if (closestNode == null || n.getDistance(indicator) < closestNode.getDistance(indicator)) {
- closestNode = n;
- }
- }
-
- return closestNode;
- }
-
- /**
- * This method creates a tree with an arbitrary fan out (two means binary tree).
- * If topology information or penalties are available, it considers that.
- * If fanout is set to 1, it creates a sequential tree.
- * if fanout is set to Integer.MAXVALUE, it creates a unicast tree.
- *
- * @param nodes
- * @param fanout
- * @return
- */
- private MulticastForwardingTable createDefaultTree(LinkedList<TreeNode> nodes, int fanout) {
-
- // Store nodes that already have a parent, but no children
- LinkedList<TreeNode> connectedNodes = new LinkedList<TreeNode>();
-
- final TreeNode rootnode = nodes.pollFirst();
- TreeNode actualnode = rootnode;
-
- while (nodes.size() > 0) { // We still have unconnected nodes...
-
- for (int i = 0; i < fanout; i++) {
-
- if (nodes.size() > 0) {
- // pick the closest one and attach to actualnode
- TreeNode child = pollClosestNode(actualnode, nodes);
- actualnode.addChild(child);
-
- // The child is now connected and can be used as forwarder in the next iteration..
- connectedNodes.add(child);
- } else {
- break;
- }
- }
-
- // OK.. take the next node to attach children to it..
- // TODO: Optimization? "pollBest()" ?
- actualnode = connectedNodes.pollFirst();
-
- }
- LOG.info("created multicast tree with following topology:\n" + rootnode.printTree());
-
- return rootnode.createForwardingTable();
-
- }
-
- /**
- * Reads a hard-coded tree topology from file and creates a tree according to the hard-coded
- * topology from the file.
- *
- * @param nodes
- * @return
- */
- private MulticastForwardingTable createHardCodedTree(LinkedList<TreeNode> nodes) {
- try {
- FileInputStream fstream = new FileInputStream(this.hardCodedTreeFilePath);
- DataInputStream in = new DataInputStream(fstream);
- BufferedReader br = new BufferedReader(new InputStreamReader(in));
- String strLine;
- while ((strLine = br.readLine()) != null) {
- String[] values = strLine.split(" ");
- String actualhostname = values[0];
- for (TreeNode n : nodes) {
- if (n.toString().equals(actualhostname)) {
- // we found the node.. connect the children
- for (int i = 1; i < values.length; i++) {
- for (TreeNode childnode : nodes) {
- if (childnode.toString().equals(values[i])) {
- n.addChild(childnode);
- }
- }
- }
- }
- }
- }
- br.close();
- // First node is root.. create tree. easy
- return nodes.getFirst().createForwardingTable();
-
- } catch (Exception e) {
- System.out.println("Error reading hard-coded topology file for multicast tree: " + e.getMessage());
- return null;
- }
- }
-
- /**
- * Checks, if all target vertices for multicast transmisison are ready. If vertices are in state ASSIGNED, it will
- * deploy those vertices.
- *
- * @param caller
- * @param jobID
- * @param sourceChannelID
- * @return
- */
- private boolean checkIfAllTargetVerticesReady(InstanceConnectionInfo caller, JobID jobID, ChannelID sourceChannelID) {
-
- final ExecutionGraph eg = this.scheduler.getExecutionGraphByID(jobID);
-
- final ExecutionEdge outputChannel = eg.getEdgeByID(sourceChannelID);
-
- final ExecutionGate broadcastGate = outputChannel.getOutputGate();
-
- List<ExecutionVertex> verticesToDeploy = null;
-
- // get all broadcast output channels
- final int numberOfOutputChannels = broadcastGate.getNumberOfEdges();
- for (int i = 0; i < numberOfOutputChannels; ++i) {
-
- final ExecutionEdge c = broadcastGate.getEdge(i);
-
- if (c.isBroadcast()) {
-
- final ExecutionVertex targetVertex = c.getInputGate().getVertex();
-
- if (targetVertex.getExecutionState() == ExecutionState.ASSIGNED) {
- if (verticesToDeploy == null) {
- verticesToDeploy = new ArrayList<ExecutionVertex>();
- }
- verticesToDeploy.add(targetVertex);
- } else {
-
- if (targetVertex.getExecutionState() != ExecutionState.RUNNING
- && targetVertex.getExecutionState() != ExecutionState.FINISHING) {
- return false;
- }
- }
- }
- }
-
- if (verticesToDeploy != null) {
- this.scheduler.deployAssignedVertices(verticesToDeploy);
- return false;
- }
-
- return true;
- }
-
- /**
- * Returns a list of (physical) Nodes (=hosts) within the multicast tree. Each node contains the local ChannelIDs,
- * records
- * must be forwarded to. The first node in the List is the only multicast sender.
- *
- * @param sourceChannelID
- * @return
- */
- private LinkedList<TreeNode> extractTreeNodes(final InstanceConnectionInfo source, final JobID jobID,
- final ChannelID sourceChannelID, final boolean randomize) {
-
- final ExecutionGraph eg = this.scheduler.getExecutionGraphByID(jobID);
-
- final ExecutionEdge outputChannel = eg.getEdgeByID(sourceChannelID);
-
- final ExecutionGate broadcastGate = outputChannel.getOutputGate();
-
- final LinkedList<ExecutionEdge> outputChannels = new LinkedList<ExecutionEdge>();
-
- // Get all broadcast output channels
- final int numberOfOutputChannels = broadcastGate.getNumberOfEdges();
- for (int i = 0; i < numberOfOutputChannels; ++i) {
- final ExecutionEdge c = broadcastGate.getEdge(i);
-
- if (c.isBroadcast()) {
- outputChannels.add(c);
- }
- }
-
- final LinkedList<TreeNode> treeNodes = new LinkedList<TreeNode>();
-
- LinkedList<ChannelID> actualLocalTargets = new LinkedList<ChannelID>();
-
- int firstConnectionID = 0;
- // search for local targets for the tree node
- for (Iterator<ExecutionEdge> iter = outputChannels.iterator(); iter.hasNext();) {
-
- final ExecutionEdge actualOutputChannel = iter.next();
-
- // the connection ID should not be needed for the root node (as it is not set as remote receiver)
- // but in order to maintain consistency, it also gets the connectionID of the first channel pointing to it
- firstConnectionID = actualOutputChannel.getConnectionID();
-
- final ExecutionVertex targetVertex = actualOutputChannel.getInputGate().getVertex();
-
- // is the target vertex running on the same instance?
- if (targetVertex.getAllocatedResource().getInstance().getInstanceConnectionInfo().equals(source)) {
-
- actualLocalTargets.add(actualOutputChannel.getInputChannelID());
- iter.remove();
- }
-
- }
-
- // create sender node (root) with source instance
- TreeNode actualNode = new TreeNode(eg.getVertexByChannelID(sourceChannelID).getAllocatedResource()
- .getInstance(), source, firstConnectionID, actualLocalTargets);
-
- treeNodes.add(actualNode);
-
- // now we have the root-node.. lets extract all other nodes
-
- LinkedList<TreeNode> receiverNodes = new LinkedList<TreeNode>();
-
- while (outputChannels.size() > 0) {
-
- final ExecutionEdge firstChannel = outputChannels.pollFirst();
-
- // each receiver nodes' endpoint is associated with the connection ID
- // of the first channel pointing to this node.
- final int connectionID = firstChannel.getConnectionID();
-
- final ExecutionVertex firstTarget = firstChannel.getInputGate().getVertex();
-
- final InstanceConnectionInfo actualInstance = firstTarget.getAllocatedResource().getInstance()
- .getInstanceConnectionInfo();
-
- actualLocalTargets = new LinkedList<ChannelID>();
-
- // add first local target
- actualLocalTargets.add(firstChannel.getInputChannelID());
-
- // now we iterate through the remaining channels to find other local targets...
- for (Iterator<ExecutionEdge> iter = outputChannels.iterator(); iter.hasNext();) {
-
- final ExecutionEdge actualOutputChannel = iter.next();
-
- final ExecutionVertex actualTarget = actualOutputChannel.getInputGate().getVertex();
-
- // is the target vertex running on the same instance?
- if (actualTarget.getAllocatedResource().getInstance().getInstanceConnectionInfo()
- .equals(actualInstance)) {
- actualLocalTargets.add(actualOutputChannel.getInputChannelID());
-
- iter.remove();
-
- }
-
- }// end for
-
- // create tree node for current instance
- actualNode = new TreeNode(firstTarget.getAllocatedResource().getInstance(), actualInstance, connectionID,
- actualLocalTargets);
-
- receiverNodes.add(actualNode);
-
- }// end while
-
- // Do we want to shuffle the receiver nodes?
- // Only randomize the receivers, as the sender (the first one) has to stay the same
- if (randomize) {
- Collections.shuffle(receiverNodes);
- } else {
- // Sort Tree Nodes according to host name..
- Collections.sort(receiverNodes);
- }
-
- treeNodes.addAll(receiverNodes);
-
- return treeNodes;
-
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TopologyInformationSupplier.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TopologyInformationSupplier.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TopologyInformationSupplier.java
deleted file mode 100644
index 0e80ba2..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TopologyInformationSupplier.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/***********************************************************************************************************************
- * 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.multicast;
-
-public class TopologyInformationSupplier {
-
-
- /**
- * Returns a float representing the network distance between two nodes.
- * @param node1
- * @param node2
- * @return
- */
- public float getNetworkDistance(String node1, String node2){
- //TODO: Implement me
- return 0;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TreeNode.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TreeNode.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TreeNode.java
deleted file mode 100644
index 12ee998..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/multicast/TreeNode.java
+++ /dev/null
@@ -1,246 +0,0 @@
-/***********************************************************************************************************************
- * 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.multicast;
-
-import java.net.InetSocketAddress;
-import java.util.LinkedList;
-
-import eu.stratosphere.nephele.instance.AbstractInstance;
-import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ConnectionInfoLookupResponse;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.RemoteReceiver;
-
-/**
- * Each physical node (instance) within a multicast tree is represented by a TreeNode object.
- * It contains the connection info for the certain node and a list of the local output channels.
- *
- */
-
-public class TreeNode implements Comparable<TreeNode> {
-
- private TreeNode parentnode = null;
-
- private final AbstractInstance instance;
-
- private final InstanceConnectionInfo nodeConnectionInfo;
-
- private final int connectionID;
-
- private final LinkedList<ChannelID> localTargets;
-
- private final LinkedList<TreeNode> children = new LinkedList<TreeNode>();
-
- private final LinkedList<IntegerProperty> properties = new LinkedList<TreeNode.IntegerProperty>();
-
- private int penalty = 0;
-
- public TreeNode(AbstractInstance instance, InstanceConnectionInfo nodeConnectionInfo, int connectionID,
- LinkedList<ChannelID> localTargets) {
- this.instance = instance;
- this.connectionID = connectionID;
- this.nodeConnectionInfo = nodeConnectionInfo;
- this.localTargets = localTargets;
- }
-
- public void setProperty(String key, int value) {
- boolean exists = false;
- for (IntegerProperty property : this.properties) {
- if (property.getKey().equals(key)) {
- property.setValue(value);
- exists = true;
- break;
- }
- }
- if (!exists) {
- this.properties.add(new IntegerProperty(key, value));
- }
- }
-
- public int getProperty(String key) {
- for (IntegerProperty property : this.properties) {
- if (property.getKey().equals(key)) {
- return property.getValue();
- }
- }
- return -1;
- }
-
- public void removeChild(TreeNode child) {
- if (this.children.contains(child)) {
- child.setParent(null);
- this.children.remove(child);
- }
- }
-
- public void addChild(TreeNode child) {
- this.children.add(child);
- child.setParent(this);
- }
-
- public LinkedList<TreeNode> getChildren() {
- return this.children;
- }
-
- public TreeNode getParent() {
- return this.parentnode;
- }
-
- private InstanceConnectionInfo getConnectionInfo() {
- return this.nodeConnectionInfo;
- }
-
- private int getConnectionID() {
- return this.connectionID;
- }
-
- private void setParent(TreeNode parent) {
- this.parentnode = parent;
- }
-
- @Override
- public int compareTo(TreeNode o) {
- return this.nodeConnectionInfo.compareTo(o.nodeConnectionInfo);
- }
-
- @Override
- public boolean equals(Object o) {
- if (o instanceof TreeNode) {
- return this.nodeConnectionInfo.equals(((TreeNode) o).nodeConnectionInfo);
- } else {
- return false;
- }
- }
-
- public int getDistance(TreeNode o) {
- return this.instance.getDistance(o.instance);
- }
-
- public String toString() {
- return this.nodeConnectionInfo.toString();
- }
-
- public int getPenalty() {
- return this.penalty;
- }
-
- public void setPenalty(int penalty) {
- this.penalty = penalty;
- }
-
- /**
- * This method should be called on the root node (sender node).
- * It traverses the Tree and returns a full forwarding table
- * including all local and remote receivers.
- *
- * @return
- */
- public MulticastForwardingTable createForwardingTable() {
- MulticastForwardingTable table = new MulticastForwardingTable();
- this.generateRecursiveForwardingTable(table);
- return table;
- }
-
- /**
- * Private recursive method to generate forwarding table
- *
- * @param table
- */
- private void generateRecursiveForwardingTable(MulticastForwardingTable table) {
-
- final ConnectionInfoLookupResponse lookupResponse = ConnectionInfoLookupResponse.createReceiverFoundAndReady();
-
- // add local targets
- for (final ChannelID i : this.localTargets) {
- lookupResponse.addLocalTarget(i);
- }
-
- // add remote targets
- for (final TreeNode n : this.children) {
-
- // Instance Connection info associated with the remote target
- final InstanceConnectionInfo ici = n.getConnectionInfo();
-
- // get the connection ID associated with the remote target endpoint
- final int icid = n.getConnectionID();
-
- final InetSocketAddress isa = new InetSocketAddress(ici.getAddress(), ici.getDataPort());
-
- lookupResponse.addRemoteTarget(new RemoteReceiver(isa, icid));
- }
-
- table.addConnectionInfo(this.nodeConnectionInfo, lookupResponse);
-
- for (final TreeNode n : this.children) {
- n.generateRecursiveForwardingTable(table);
- }
- }
-
- /**
- * Prints the tree in a human readable format, starting with the actual node as root.
- *
- * @return
- */
- public String printTree() {
-
- StringBuilder sb = new StringBuilder();
- this.printRecursiveTree(sb);
- return sb.toString();
- }
-
- private void printRecursiveTree(StringBuilder sb) {
-
- if (this.children.size() > 0) {
- sb.append("STRUCT ");
-
- sb.append(this.nodeConnectionInfo);
-
- for (TreeNode n : this.children) {
- sb.append(" ");
- sb.append(n.getConnectionInfo().toString());
- }
-
- sb.append("\n");
-
- for (TreeNode n : this.children) {
- n.printRecursiveTree(sb);
- }
- }
- }
-
- private static class IntegerProperty {
-
- private String key = null;
-
- private int value = 0;
-
- public IntegerProperty(final String key, final int value) {
- this.key = key;
- this.value = value;
- }
-
- public String getKey() {
- return this.key;
- }
-
- public int getValue() {
- return this.value;
- }
-
- public void setValue(final int value) {
- this.value = value;
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java
index b1d6ff6..302192b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/TaskManagerProfiler.java
@@ -15,7 +15,7 @@ package eu.stratosphere.nephele.profiling;
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
-import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask;
+import eu.stratosphere.nephele.taskmanager.Task;
/**
* This interface must be implemented by profiling components
@@ -32,8 +32,7 @@ public interface TaskManagerProfiler {
* @param jobConfiguration
* the job configuration sent with the task
*/
- void registerExecutionListener(RuntimeTask task, Configuration jobConfiguration);
-
+ void registerExecutionListener(Task task, Configuration jobConfiguration);
/**
* Unregisters all previously register {@link ExecutionListener} objects for
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java
index 49fa8d1..58f0f38 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/TaskManagerProfilerImpl.java
@@ -13,20 +13,6 @@
package eu.stratosphere.nephele.profiling.impl;
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.lang.management.ThreadMXBean;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Timer;
-import java.util.TimerTask;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.configuration.GlobalConfiguration;
import eu.stratosphere.nephele.execution.Environment;
@@ -40,8 +26,21 @@ import eu.stratosphere.nephele.profiling.TaskManagerProfiler;
import eu.stratosphere.nephele.profiling.impl.types.InternalExecutionVertexThreadProfilingData;
import eu.stratosphere.nephele.profiling.impl.types.InternalInstanceProfilingData;
import eu.stratosphere.nephele.profiling.impl.types.ProfilingDataContainer;
-import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask;
+import eu.stratosphere.nephele.taskmanager.Task;
import eu.stratosphere.util.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.lang.management.ThreadMXBean;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Timer;
+import java.util.TimerTask;
public class TaskManagerProfilerImpl extends TimerTask implements TaskManagerProfiler {
@@ -99,7 +98,7 @@ public class TaskManagerProfilerImpl extends TimerTask implements TaskManagerPro
@Override
- public void registerExecutionListener(final RuntimeTask task, final Configuration jobConfiguration) {
+ public void registerExecutionListener(final Task task, final Configuration jobConfiguration) {
// Register profiling hook for the environment
task.registerExecutionListener(new EnvironmentListenerImpl(this, task.getRuntimeEnvironment()));
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ChannelLookupProtocol.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ChannelLookupProtocol.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ChannelLookupProtocol.java
index 3f1d669..8fdaf55 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ChannelLookupProtocol.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ChannelLookupProtocol.java
@@ -17,9 +17,9 @@ import java.io.IOException;
import eu.stratosphere.core.protocols.VersionedProtocol;
import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ConnectionInfoLookupResponse;
+import eu.stratosphere.runtime.io.network.ConnectionInfoLookupResponse;
/**
* The channel lookup protocol can be used to resolve the ID of an output channel to all recipients which shall receive
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ExtendedManagementProtocol.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ExtendedManagementProtocol.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ExtendedManagementProtocol.java
index a2578a5..461c797 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ExtendedManagementProtocol.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/ExtendedManagementProtocol.java
@@ -24,7 +24,6 @@ import eu.stratosphere.nephele.instance.InstanceType;
import eu.stratosphere.nephele.instance.InstanceTypeDescription;
import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.nephele.managementgraph.ManagementGraph;
-import eu.stratosphere.nephele.managementgraph.ManagementVertexID;
import eu.stratosphere.nephele.topology.NetworkTopology;
/**
@@ -82,18 +81,6 @@ public interface ExtendedManagementProtocol extends JobManagementProtocol {
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).
*
* @param instanceName
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
index 63509ab..19522db 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/protocols/TaskOperationProtocol.java
@@ -23,9 +23,8 @@ import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileRequest
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheProfileResponse;
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheUpdate;
import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
-import eu.stratosphere.nephele.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.nephele.taskmanager.TaskCancelResult;
-import eu.stratosphere.nephele.taskmanager.TaskKillResult;
import eu.stratosphere.nephele.taskmanager.TaskSubmissionResult;
/**
@@ -59,17 +58,6 @@ public interface TaskOperationProtocol extends VersionedProtocol {
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}
* object.
*
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/iomanager/BlockChannelAccess.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/iomanager/BlockChannelAccess.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/iomanager/BlockChannelAccess.java
index be24922..8fdaa5d 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/iomanager/BlockChannelAccess.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/iomanager/BlockChannelAccess.java
@@ -134,9 +134,7 @@ public abstract class BlockChannelAccess<R extends IORequest, C extends Collecti
this.closeLock.wait(1000);
checkErroneous();
}
- catch (InterruptedException iex) {
- throw new IOException("Block channel access was interrupted while closing.");
- }
+ catch (InterruptedException iex) {}
}
}
finally {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/MemoryManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/MemoryManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/MemoryManager.java
index 7680843..a8fe096 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/MemoryManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/MemoryManager.java
@@ -56,8 +56,7 @@ public interface MemoryManager {
/**
* Releases all memory segments for the given task.
- *
- * @param <T> The type of memory segment.
+ *
* @param task The task whose memory segments are to be released.
*/
void releaseAll(AbstractInvokable task);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/spi/DefaultMemoryManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/spi/DefaultMemoryManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/spi/DefaultMemoryManager.java
index 5a74bab..8bc7b13 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/spi/DefaultMemoryManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/spi/DefaultMemoryManager.java
@@ -13,7 +13,6 @@
package eu.stratosphere.nephele.services.memorymanager.spi;
-
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collection;
@@ -125,7 +124,6 @@ public class DefaultMemoryManager implements MemoryManager {
}
}
-
@Override
public void shutdown() {
// -------------------- BEGIN CRITICAL SECTION -------------------
@@ -150,7 +148,6 @@ public class DefaultMemoryManager implements MemoryManager {
}
// -------------------- END CRITICAL SECTION -------------------
}
-
public boolean verifyEmpty() {
synchronized (this.lock) {
@@ -161,7 +158,7 @@ public class DefaultMemoryManager implements MemoryManager {
// ------------------------------------------------------------------------
// MemoryManager interface implementation
// ------------------------------------------------------------------------
-
+
@Override
public List<MemorySegment> allocatePages(AbstractInvokable owner, int numPages) throws MemoryAllocationException {
final ArrayList<MemorySegment> segs = new ArrayList<MemorySegment>(numPages);
@@ -212,7 +209,6 @@ public class DefaultMemoryManager implements MemoryManager {
}
// ------------------------------------------------------------------------
-
@Override
public void release(MemorySegment segment) {
@@ -254,7 +250,6 @@ public class DefaultMemoryManager implements MemoryManager {
// -------------------- END CRITICAL SECTION -------------------
}
-
@Override
public <T extends MemorySegment> void release(Collection<T> segments) {
@@ -317,7 +312,6 @@ public class DefaultMemoryManager implements MemoryManager {
// -------------------- END CRITICAL SECTION -------------------
}
-
@Override
public void releaseAll(AbstractInvokable owner) {
// -------------------- BEGIN CRITICAL SECTION -------------------
@@ -326,28 +320,27 @@ public class DefaultMemoryManager implements MemoryManager {
if (this.isShutDown) {
throw new IllegalStateException("Memory manager has been shut down.");
}
-
+
// get all segments
final Set<DefaultMemorySegment> segments = this.allocatedSegments.remove(owner);
-
+
// all segments may have been freed previously individually
if (segments == null || segments.isEmpty()) {
return;
}
-
+
// free each segment
for (DefaultMemorySegment seg : segments) {
final byte[] buffer = seg.destroy();
this.freeSegments.add(buffer);
}
-
+
segments.clear();
}
// -------------------- END CRITICAL SECTION -------------------
}
// ------------------------------------------------------------------------
-
@Override
public int getPageSize() {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/ExecutorThreadFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/ExecutorThreadFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/ExecutorThreadFactory.java
new file mode 100644
index 0000000..bfad346
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/ExecutorThreadFactory.java
@@ -0,0 +1,35 @@
+/***********************************************************************************************************************
+ * 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;
+
+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;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java
index df9bbde..06eec0c 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/Task.java
@@ -15,42 +15,92 @@ package eu.stratosphere.nephele.taskmanager;
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.nephele.execution.Environment;
+import eu.stratosphere.nephele.execution.ExecutionListener;
+import eu.stratosphere.nephele.execution.ExecutionObserver;
import eu.stratosphere.nephele.execution.ExecutionState;
+import eu.stratosphere.nephele.execution.ExecutionStateTransition;
+import eu.stratosphere.nephele.execution.RuntimeEnvironment;
import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.nephele.profiling.TaskManagerProfiler;
import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher;
+import eu.stratosphere.nephele.template.AbstractInvokable;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.util.Iterator;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+public final class Task implements ExecutionObserver {
+
+ /**
+ * The log object used for debugging.
+ */
+ private static final Log LOG = LogFactory.getLog(Task.class);
+
+ private final ExecutionVertexID vertexID;
+
+ private final RuntimeEnvironment environment;
+
+ private final TaskManager taskManager;
+
+ /**
+ * Stores whether the task has been canceled.
+ */
+ private volatile boolean isCanceled = false;
+
+ /**
+ * The current execution state of the task
+ */
+ private volatile ExecutionState executionState = ExecutionState.STARTING;
+
+ private Queue<ExecutionListener> registeredListeners = new ConcurrentLinkedQueue<ExecutionListener>();
+
+ public Task(final ExecutionVertexID vertexID, final RuntimeEnvironment environment,
+ final TaskManager taskManager) {
+
+ this.vertexID = vertexID;
+ this.environment = environment;
+ this.taskManager = taskManager;
+
+ this.environment.setExecutionObserver(this);
+ }
-public interface Task {
/**
* Returns the ID of the job this task belongs to.
*
* @return the ID of the job this task belongs to
*/
- JobID getJobID();
+ public JobID getJobID() {
+ return this.environment.getJobID();
+ }
/**
* Returns the ID of this task.
*
* @return the ID of this task
*/
- ExecutionVertexID getVertexID();
+ public ExecutionVertexID getVertexID() {
+ return this.vertexID;
+ }
/**
* Returns the environment associated with this task.
*
* @return the environment associated with this task
*/
- Environment getEnvironment();
+ public Environment getEnvironment() {
+ return this.environment;
+ }
/**
* Marks the task as failed and triggers the appropriate state changes.
*/
- void markAsFailed();
+ public void markAsFailed() {
+ executionStateChanged(ExecutionState.FAILED, "Execution thread died unexpectedly");
+ }
/**
* Checks if the state of the thread which is associated with this task is <code>TERMINATED</code>.
@@ -58,22 +108,72 @@ public interface Task {
* @return <code>true</code> if the state of this thread which is associated with this task is
* <code>TERMINATED</code>, <code>false</code> otherwise
*/
- boolean isTerminated();
+ 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.
*/
- void startExecution();
+ public void startExecution() {
+
+ final Thread thread = this.environment.getExecutingThread();
+ thread.start();
+ }
/**
* Cancels the execution of the task (i.e. interrupts the execution thread).
*/
- void cancelExecution();
+ public void cancelExecution() {
+ final Thread executingThread = this.environment.getExecutingThread();
+
+ if (executingThread == null) {
+ return;
+ }
+
+ LOG.info("Canceling " + this.environment.getTaskNameWithIndex());
+
+ this.isCanceled = true;
+ // Change state
+ executionStateChanged(ExecutionState.CANCELING, null);
+
+ // Request user code to shut down
+ try {
+ final AbstractInvokable invokable = this.environment.getInvokable();
+ if (invokable != null) {
+ invokable.cancel();
+ }
+ } catch (Throwable e) {
+ LOG.error("Error while canceling task", e);
+ }
+
+ // Continuously interrupt the user thread until it changed to state CANCELED
+ while (true) {
+ executingThread.interrupt();
+
+ if (!executingThread.isAlive()) {
+ break;
+ }
+
+ try {
+ executingThread.join(1000);
+ } catch (InterruptedException e) {}
+
+ if (!executingThread.isAlive()) {
+ break;
+ }
+
+ if (LOG.isDebugEnabled())
+ LOG.debug("Sending repeated canceling signal to " +
+ this.environment.getTaskName() + " with state " + this.executionState);
+ }
+ }
- /**
- * Kills the task (i.e. interrupts the execution thread).
- */
- void killExecution();
/**
* Registers the task manager profiler with the task.
@@ -83,7 +183,9 @@ public interface Task {
* @param jobConfiguration
* the configuration attached to the job
*/
- void registerProfiler(TaskManagerProfiler taskManagerProfiler, Configuration jobConfiguration);
+ public void registerProfiler(final TaskManagerProfiler taskManagerProfiler, final Configuration jobConfiguration) {
+ taskManagerProfiler.registerExecutionListener(this, jobConfiguration);
+ }
/**
* Unregisters the task from the central memory manager.
@@ -91,7 +193,11 @@ public interface Task {
* @param memoryManager
* the central memory manager
*/
- void unregisterMemoryManager(MemoryManager memoryManager);
+ public void unregisterMemoryManager(final MemoryManager memoryManager) {
+ if (memoryManager != null) {
+ memoryManager.releaseAll(this.environment.getInvokable());
+ }
+ }
/**
* Unregisters the task from the task manager profiler.
@@ -99,15 +205,124 @@ public interface Task {
* @param taskManagerProfiler
* the task manager profiler
*/
- void unregisterProfiler(TaskManagerProfiler taskManagerProfiler);
+ public void unregisterProfiler(final TaskManagerProfiler taskManagerProfiler) {
+ if (taskManagerProfiler != null) {
+ taskManagerProfiler.unregisterExecutionListener(this.vertexID);
+ }
+ }
/**
* Returns the current execution state of the task.
*
* @return the current execution state of the task
*/
- ExecutionState getExecutionState();
+ public ExecutionState getExecutionState() {
+ return this.executionState;
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+ // ExecutionObserver methods
+ // -----------------------------------------------------------------------------------------------------------------
+ @Override
+ public void executionStateChanged(final ExecutionState newExecutionState, final String optionalMessage) {
+
+ // Check the state transition
+ ExecutionStateTransition.checkTransition(false, getTaskName(), this.executionState, newExecutionState);
+
+ // Make sure the reason for a transition to FAILED appears in the log files
+ if (newExecutionState == ExecutionState.FAILED) {
+ LOG.error(optionalMessage);
+ }
+
+ // Notify all listener objects
+ final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
+ while (it.hasNext()) {
+ it.next().executionStateChanged(this.environment.getJobID(), this.vertexID, newExecutionState,
+ optionalMessage);
+ }
+
+ // Store the new execution state
+ this.executionState = newExecutionState;
+
+ // Finally propagate the state change to the job manager
+ this.taskManager.executionStateChanged(this.environment.getJobID(), this.vertexID, newExecutionState,
+ optionalMessage);
+ }
+
+ /**
+ * Returns the name of the task associated with this observer object.
+ *
+ * @return the name of the task associated with this observer object
+ */
+ private String getTaskName() {
+
+ return this.environment.getTaskName() + " (" + (this.environment.getIndexInSubtaskGroup() + 1) + "/"
+ + this.environment.getCurrentNumberOfSubtasks() + ")";
+ }
+
+
+ @Override
+ public void userThreadStarted(final Thread userThread) {
+
+ // Notify the listeners
+ final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
+ while (it.hasNext()) {
+ it.next().userThreadStarted(this.environment.getJobID(), this.vertexID, userThread);
+ }
+ }
+
+
+ @Override
+ public void userThreadFinished(final Thread userThread) {
+
+ // Notify the listeners
+ final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
+ while (it.hasNext()) {
+ it.next().userThreadFinished(this.environment.getJobID(), this.vertexID, userThread);
+ }
+ }
+
+ /**
+ * Registers the {@link ExecutionListener} object for this task. This object
+ * will be notified about important events during the task execution.
+ *
+ * @param executionListener
+ * the object to be notified for important events during the task execution
+ */
+
+ public void registerExecutionListener(final ExecutionListener executionListener) {
+
+ this.registeredListeners.add(executionListener);
+ }
+
+ /**
+ * Unregisters the {@link ExecutionListener} object for this environment. This object
+ * will no longer be notified about important events during the task execution.
+ *
+ * @param executionListener
+ * the lister object to be unregistered
+ */
+
+ public void unregisterExecutionListener(final ExecutionListener executionListener) {
+
+ this.registeredListeners.remove(executionListener);
+ }
+
+
+ @Override
+ public boolean isCanceled() {
+
+ return this.isCanceled;
+ }
+
+ /**
+ * Returns the runtime environment associated with this task.
+ *
+ * @return the runtime environment associated with this task
+ */
+ public RuntimeEnvironment getRuntimeEnvironment() {
+
+ return this.environment;
+ }
- TaskContext createTaskContext(TransferEnvelopeDispatcher transferEnvelopeDispatcher,
- LocalBufferPoolOwner previousBufferPoolOwner);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskKillResult.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskKillResult.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskKillResult.java
deleted file mode 100644
index 3c0002c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskKillResult.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/***********************************************************************************************************************
- * 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;
-
-import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
-
-/**
- * A <code>TaskKillResult</code> is used to report the results
- * 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 class TaskKillResult extends AbstractTaskResult {
-
- /**
- * Constructs a new task kill result.
- *
- * @param vertexID
- * the task ID this result belongs to
- * @param returnCode
- * the return code of the kill
- */
- public TaskKillResult(final ExecutionVertexID vertexID, final ReturnCode returnCode) {
- super(vertexID, returnCode);
- }
-
- /**
- * Constructs an empty task kill result.
- */
- public TaskKillResult() {
- super();
- }
-}
[14/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/Gate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/Gate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/Gate.java
new file mode 100644
index 0000000..c9db615
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/Gate.java
@@ -0,0 +1,174 @@
+/***********************************************************************************************************************
+ * 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.gates;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.nephele.event.task.EventListener;
+import eu.stratosphere.nephele.event.task.EventNotificationManager;
+import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.JobID;
+
+import java.io.IOException;
+
+/**
+ * In Nephele a gate represents the connection between a user program and the processing framework. A gate
+ * must be connected to exactly one record reader/writer and to at least one channel. The <code>Gate</code> class itself
+ * is abstract. A gate automatically created for every record reader/writer in the user program. A gate can only be used
+ * to transport one specific type of records.
+ * <p>
+ * This class in general is not thread-safe.
+ *
+ * @param <T>
+ * the record type to be transported from this gate
+ *
+ * TODO refactor with changes to input side
+ */
+public abstract class Gate<T extends IOReadableWritable> {
+
+ /**
+ * The ID of the job this gate belongs to.
+ */
+ private final JobID jobID;
+
+ /**
+ * The ID of this gate.
+ */
+ private final GateID gateID;
+
+ /**
+ * The index of the gate in the list of available input/output gates.
+ */
+ private final int index;
+
+ /**
+ * The event notification manager used to dispatch events.
+ */
+ private final EventNotificationManager eventNotificationManager = new EventNotificationManager();
+
+ /**
+ * The type of input/output channels connected to this gate.
+ */
+ private ChannelType channelType = ChannelType.NETWORK;
+
+ /**
+ * Constructs a new abstract gate
+ *
+ * @param jobID
+ * the ID of the job this gate belongs to
+ * @param gateID
+ * the ID of this gate
+ * @param index
+ * the index of the gate in the list of available input/output gates.
+ */
+ protected Gate(final JobID jobID, final GateID gateID, final int index) {
+ this.jobID = jobID;
+ this.gateID = gateID;
+ this.index = index;
+ }
+
+ public final int getIndex() {
+ return this.index;
+ }
+
+ /**
+ * Returns the event notification manager used to dispatch events.
+ *
+ * @return the event notification manager used to dispatch events
+ */
+ protected final EventNotificationManager getEventNotificationManager() {
+ return this.eventNotificationManager;
+ }
+
+ public String toString() {
+
+ return "Gate " + this.index;
+ }
+
+ public final void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
+
+ this.eventNotificationManager.subscribeToEvent(eventListener, eventType);
+ }
+
+ public final void unsubscribeFromEvent(final EventListener eventListener,
+ final Class<? extends AbstractTaskEvent> eventType) {
+
+ this.eventNotificationManager.unsubscribeFromEvent(eventListener, eventType);
+ }
+
+ public final void deliverEvent(final AbstractTaskEvent event) {
+
+ this.eventNotificationManager.deliverEvent((AbstractTaskEvent) event);
+ }
+
+ public final void setChannelType(final ChannelType channelType) {
+
+ this.channelType = channelType;
+ }
+
+ public final ChannelType getChannelType() {
+
+ return this.channelType;
+ }
+
+ public JobID getJobID() {
+
+ return this.jobID;
+ }
+
+ public GateID getGateID() {
+
+ return this.gateID;
+ }
+
+ // FROM GATE INTERFACE
+
+ /**
+ * Publishes an event.
+ *
+ * @param event
+ * the event to be published
+ * @throws IOException
+ * thrown if an error occurs while transmitting the event
+ * @throws InterruptedException
+ * thrown if the thread is interrupted while waiting for the event to be published
+ */
+ abstract public void publishEvent(AbstractEvent event) throws IOException, InterruptedException;
+
+ /**
+ * Releases the allocated resources (particularly buffer) of all channels attached to this gate. This method
+ * should only be called after the respected task has stopped running.
+ */
+ abstract public void releaseAllChannelResources();
+
+ /**
+ * Checks if the gate is closed. The gate is closed if all this associated channels are closed.
+ *
+ * @return <code>true</code> if the gate is closed, <code>false</code> otherwise
+ * @throws IOException
+ * thrown if any error occurred while closing the gate
+ * @throws InterruptedException
+ * thrown if the gate is interrupted while waiting for this operation to complete
+ */
+ abstract public boolean isClosed() throws IOException, InterruptedException;
+
+ /**
+ * Checks if the considered gate is an input gate.
+ *
+ * @return <code>true</code> if the considered gate is an input gate, <code>false</code> if it is an output gate
+ */
+ abstract public boolean isInputGate();
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/GateID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/GateID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/GateID.java
new file mode 100644
index 0000000..8375c88
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/GateID.java
@@ -0,0 +1,24 @@
+/***********************************************************************************************************************
+ * 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.gates;
+
+import eu.stratosphere.nephele.AbstractID;
+
+/**
+ * A class for statistically unique gate IDs.
+ *
+ */
+public final class GateID extends AbstractID {
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputChannelResult.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputChannelResult.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputChannelResult.java
new file mode 100644
index 0000000..e2083e5
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputChannelResult.java
@@ -0,0 +1,23 @@
+/***********************************************************************************************************************
+ * 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.gates;
+
+public enum InputChannelResult {
+
+ NONE,
+ INTERMEDIATE_RECORD_FROM_BUFFER,
+ LAST_RECORD_FROM_BUFFER,
+ END_OF_SUPERSTEP,
+ TASK_EVENT,
+ END_OF_STREAM;
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputGate.java
new file mode 100644
index 0000000..bdac7a2
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/InputGate.java
@@ -0,0 +1,384 @@
+/***********************************************************************************************************************
+ * 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.gates;
+
+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.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicReference;
+
+import eu.stratosphere.nephele.deployment.ChannelDeploymentDescriptor;
+import eu.stratosphere.nephele.deployment.GateDeploymentDescriptor;
+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.BufferProvider;
+import eu.stratosphere.runtime.io.network.bufferprovider.GlobalBufferPool;
+import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPool;
+import eu.stratosphere.runtime.io.network.bufferprovider.LocalBufferPoolOwner;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
+import eu.stratosphere.nephele.execution.Environment;
+import eu.stratosphere.runtime.io.channels.InputChannel;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.nephele.jobgraph.JobID;
+
+/**
+ * In Nephele input gates are a specialization of general gates and connect input channels and record readers. As
+ * channels, input gates are always parameterized to a specific type of record which they can transport. In contrast to
+ * output gates input gates can be associated with a {@link eu.stratosphere.runtime.io.serialization.io.DistributionPattern} object which dictates the concrete
+ * wiring between two groups of vertices.
+ *
+ * @param <T> The type of record that can be transported through this gate.
+ */
+public class InputGate<T extends IOReadableWritable> extends Gate<T> implements BufferProvider, LocalBufferPoolOwner {
+
+ /**
+ * The log object used for debugging.
+ */
+ private static final Log LOG = LogFactory.getLog(InputGate.class);
+
+ /**
+ * The array of input channels attached to this input gate.
+ */
+ private InputChannel<T>[] channels;
+
+ /**
+ * Queue with indices of channels that store at least one available record.
+ */
+ private final BlockingQueue<Integer> availableChannels = new LinkedBlockingQueue<Integer>();
+
+ /**
+ * The listener object to be notified when a channel has at least one record available.
+ */
+ private final AtomicReference<RecordAvailabilityListener<T>> recordAvailabilityListener = new AtomicReference<RecordAvailabilityListener<T>>(null);
+
+
+ private AbstractTaskEvent currentEvent;
+
+ /**
+ * If the value of this variable is set to <code>true</code>, the input gate is closed.
+ */
+ private boolean isClosed = false;
+
+ /**
+ * The channel to read from next.
+ */
+ private int channelToReadFrom = -1;
+
+ private LocalBufferPool bufferPool;
+
+ /**
+ * Constructs a new runtime input gate.
+ *
+ * @param jobID
+ * the ID of the job this input gate belongs to
+ * @param gateID
+ * the ID of the gate
+ * @param index
+ * the index assigned to this input gate at the {@link Environment} object
+ */
+ public InputGate(final JobID jobID, final GateID gateID, final int index) {
+ super(jobID, gateID, index);
+ }
+
+ public void initializeChannels(GateDeploymentDescriptor inputGateDescriptor){
+ channels = new InputChannel[inputGateDescriptor.getNumberOfChannelDescriptors()];
+
+ setChannelType(inputGateDescriptor.getChannelType());
+
+ final int nicdd = inputGateDescriptor.getNumberOfChannelDescriptors();
+
+ for(int i = 0; i < nicdd; i++){
+ final ChannelDeploymentDescriptor cdd = inputGateDescriptor.getChannelDescriptor(i);
+ channels[i] = new InputChannel<T>(this, i, cdd.getInputChannelID(),
+ cdd.getOutputChannelID(), getChannelType());
+ }
+ }
+
+ @Override
+ public boolean isInputGate() {
+ return true;
+ }
+
+ /**
+ * Returns the number of input channels associated with this input gate.
+ *
+ * @return the number of input channels associated with this input gate
+ */
+ public int getNumberOfInputChannels() {
+ return this.channels.length;
+ }
+
+ /**
+ * Returns the input channel from position <code>pos</code> of the gate's internal channel list.
+ *
+ * @param pos
+ * the position to retrieve the channel from
+ * @return the channel from the given position or <code>null</code> if such position does not exist.
+ */
+ public InputChannel<T> getInputChannel(int pos) {
+ return this.channels[pos];
+ }
+
+ public InputChannel<T>[] channels() {
+ return this.channels;
+ }
+
+ /**
+ * Reads a record from one of the associated input channels. Channels are read such that one buffer from a channel is
+ * consecutively consumed. The buffers in turn are consumed in the order in which they arrive.
+ * Note that this method is not guaranteed to return a record, because the currently available channel data may not always
+ * constitute an entire record, when events or partial records are part of the data.
+ *
+ * When called even though no data is available, this call will block until data is available, so this method should be called
+ * when waiting is desired (such as when synchronously consuming a single gate) or only when it is known that data is available
+ * (such as when reading a union of multiple input gates).
+ *
+ * @param target The record object into which to construct the complete record.
+ * @return The result indicating whether a complete record is available, a event is available, only incomplete data
+ * is available (NONE), or the gate is exhausted.
+ * @throws IOException Thrown when an error occurred in the network stack relating to this channel.
+ * @throws InterruptedException Thrown, when the thread working on this channel is interrupted.
+ */
+ public InputChannelResult readRecord(T target) throws IOException, InterruptedException {
+
+ if (this.channelToReadFrom == -1) {
+ if (this.isClosed()) {
+ return InputChannelResult.END_OF_STREAM;
+ }
+
+ if (Thread.interrupted()) {
+ throw new InterruptedException();
+ }
+
+ this.channelToReadFrom = waitForAnyChannelToBecomeAvailable();
+ }
+
+ InputChannelResult result = this.getInputChannel(this.channelToReadFrom).readRecord(target);
+ switch (result) {
+ case INTERMEDIATE_RECORD_FROM_BUFFER: // full record and we can stay on the same channel
+ return InputChannelResult.INTERMEDIATE_RECORD_FROM_BUFFER;
+
+ case LAST_RECORD_FROM_BUFFER: // full record, but we must switch the channel afterwards
+ this.channelToReadFrom = -1;
+ return InputChannelResult.LAST_RECORD_FROM_BUFFER;
+
+ case END_OF_SUPERSTEP:
+ this.channelToReadFrom = -1;
+ return InputChannelResult.END_OF_SUPERSTEP;
+
+ case TASK_EVENT: // task event
+ this.currentEvent = this.getInputChannel(this.channelToReadFrom).getCurrentEvent();
+ this.channelToReadFrom = -1; // event always marks a unit as consumed
+ return InputChannelResult.TASK_EVENT;
+
+ case NONE: // internal event or an incomplete record that needs further chunks
+ // the current unit is exhausted
+ this.channelToReadFrom = -1;
+ return InputChannelResult.NONE;
+
+ case END_OF_STREAM: // channel is done
+ this.channelToReadFrom = -1;
+ return isClosed() ? InputChannelResult.END_OF_STREAM : InputChannelResult.NONE;
+
+ default: // silence the compiler
+ throw new RuntimeException();
+ }
+ }
+
+ public AbstractTaskEvent getCurrentEvent() {
+ AbstractTaskEvent e = this.currentEvent;
+ this.currentEvent = null;
+ return e;
+ }
+
+ /**
+ * Notify the gate that the channel with the given index has
+ * at least one record available.
+ *
+ * @param channelIndex
+ * the index of the channel which has at least one record available
+ */
+ public void notifyRecordIsAvailable(int channelIndex) {
+ this.availableChannels.add(Integer.valueOf(channelIndex));
+
+ RecordAvailabilityListener<T> listener = this.recordAvailabilityListener.get();
+ if (listener != null) {
+ listener.reportRecordAvailability(this);
+ }
+ }
+
+ /**
+ * This method returns the index of a channel which has at least
+ * one record available. The method may block until at least one
+ * channel has become ready.
+ *
+ * @return the index of the channel which has at least one record available
+ */
+ public int waitForAnyChannelToBecomeAvailable() throws InterruptedException {
+ return this.availableChannels.take().intValue();
+ }
+
+
+ @Override
+ public boolean isClosed() throws IOException, InterruptedException {
+
+ if (this.isClosed) {
+ return true;
+ }
+
+ for (int i = 0; i < this.getNumberOfInputChannels(); i++) {
+ final InputChannel<T> inputChannel = this.channels[i];
+ if (!inputChannel.isClosed()) {
+ return false;
+ }
+ }
+
+ this.isClosed = true;
+
+ return true;
+ }
+
+
+ /**
+ * Immediately closes the input gate and all its input channels. The corresponding
+ * output channels are notified. Any remaining records in any buffers or queue is considered
+ * irrelevant and is discarded.
+ *
+ * @throws IOException
+ * thrown if an I/O error occurs while closing the gate
+ * @throws InterruptedException
+ * thrown if the thread is interrupted while waiting for the gate to be closed
+ */
+ public void close() throws IOException, InterruptedException {
+
+ for (int i = 0; i < this.getNumberOfInputChannels(); i++) {
+ final InputChannel<T> inputChannel = this.channels[i];
+ inputChannel.close();
+ }
+
+ }
+
+
+ @Override
+ public String toString() {
+ return "Input " + super.toString();
+ }
+
+
+ @Override
+ public void publishEvent(AbstractEvent event) throws IOException, InterruptedException {
+
+ // Copy event to all connected channels
+ for(int i=0; i< getNumberOfChannels(); i++){
+ channels[i].transferEvent(event);
+ }
+ }
+
+
+ @Override
+ public void releaseAllChannelResources() {
+
+ for(int i=0; i< getNumberOfChannels(); i++){
+ channels[i].releaseAllResources();
+ }
+ }
+
+ /**
+ * Registers a {@link RecordAvailabilityListener} with this input gate.
+ *
+ * @param listener
+ * the listener object to be registered
+ */
+ public void registerRecordAvailabilityListener(final RecordAvailabilityListener<T> listener) {
+ if (!this.recordAvailabilityListener.compareAndSet(null, listener)) {
+ throw new IllegalStateException(this.recordAvailabilityListener
+ + " is already registered as a record availability listener");
+ }
+ }
+
+ /**
+ * Notify the gate that is has consumed a data unit from the channel with the given index
+ *
+ * @param channelIndex
+ * the index of the channel from which a data unit has been consumed
+ */
+ public void notifyDataUnitConsumed(int channelIndex) {
+ this.channelToReadFrom = -1;
+ }
+
+ //
+
+ @Override
+ public Buffer requestBuffer(int minBufferSize) throws IOException {
+ return this.bufferPool.requestBuffer(minBufferSize);
+ }
+
+ @Override
+ public Buffer requestBufferBlocking(int minBufferSize) throws IOException, InterruptedException {
+ return this.bufferPool.requestBufferBlocking(minBufferSize);
+ }
+
+ @Override
+ public int getBufferSize() {
+ return this.bufferPool.getBufferSize();
+ }
+
+ @Override
+ public int getNumberOfChannels() {
+ return getNumberOfInputChannels();
+ }
+
+ @Override
+ public void setDesignatedNumberOfBuffers(int numBuffers) {
+ this.bufferPool.setNumDesignatedBuffers(numBuffers);
+ }
+
+ @Override
+ public void clearLocalBufferPool() {
+ this.bufferPool.destroy();
+ }
+
+ @Override
+ public void registerGlobalBufferPool(GlobalBufferPool globalBufferPool) {
+ this.bufferPool = new LocalBufferPool(globalBufferPool, 1);
+ }
+
+ @Override
+ public void logBufferUtilization() {
+ LOG.info(String.format("\t%s: %d available, %d requested, %d designated",
+ this,
+ this.bufferPool.numAvailableBuffers(),
+ this.bufferPool.numRequestedBuffers(),
+ this.bufferPool.numDesignatedBuffers()));
+ }
+
+ @Override
+ public void reportAsynchronousEvent() {
+ this.bufferPool.reportAsynchronousEvent();
+ }
+
+ @Override
+ public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+ return this.bufferPool.registerBufferAvailabilityListener(listener);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java
new file mode 100644
index 0000000..d3eaea1
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/OutputGate.java
@@ -0,0 +1,165 @@
+/***********************************************************************************************************************
+ * 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.gates;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.deployment.ChannelDeploymentDescriptor;
+import eu.stratosphere.nephele.deployment.GateDeploymentDescriptor;
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.OutputChannel;
+import eu.stratosphere.nephele.jobgraph.JobID;
+
+import java.io.IOException;
+
+public class OutputGate extends Gate<IOReadableWritable> {
+
+ private OutputChannel[] channels;
+
+ private boolean closed;
+
+ /**
+ * Constructs a new output gate.
+ *
+ * @param jobId the ID of the job this input gate belongs to
+ * @param gateId the ID of the gate
+ * @param index the index assigned to this output gate at the Environment object
+ */
+ public OutputGate(JobID jobId, GateID gateId, int index) {
+ super(jobId, gateId, index);
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+ // Data processing
+ // -----------------------------------------------------------------------------------------------------------------
+
+ public void sendBuffer(Buffer buffer, int targetChannel) throws IOException, InterruptedException {
+ this.channels[targetChannel].sendBuffer(buffer);
+ }
+
+ public void sendEvent(AbstractEvent event, int targetChannel) throws IOException, InterruptedException {
+ this.channels[targetChannel].sendEvent(event);
+ }
+
+ public void sendBufferAndEvent(Buffer buffer, AbstractEvent event, int targetChannel) throws IOException, InterruptedException {
+ this.channels[targetChannel].sendBufferAndEvent(buffer, event);
+ }
+
+ public void broadcastBuffer(Buffer buffer) throws IOException, InterruptedException {
+ for (int i = 1; i < this.channels.length; i++) {
+ channels[i].sendBuffer(buffer.duplicate());
+ }
+ channels[0].sendBuffer(buffer);
+ }
+
+ public void broadcastEvent(AbstractEvent event) throws IOException, InterruptedException {
+ for (OutputChannel channel : this.channels) {
+ channel.sendEvent(event);
+ }
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+ // Channels
+ // -----------------------------------------------------------------------------------------------------------------
+
+ public void initializeChannels(GateDeploymentDescriptor descriptor) {
+ int numChannels = descriptor.getNumberOfChannelDescriptors();
+ this.channels = new OutputChannel[numChannels];
+
+ setChannelType(descriptor.getChannelType());
+
+ for (int i = 0; i < numChannels; i++) {
+ ChannelDeploymentDescriptor channelDescriptor = descriptor.getChannelDescriptor(i);
+
+ ChannelID id = channelDescriptor.getOutputChannelID();
+ ChannelID connectedId = channelDescriptor.getInputChannelID();
+
+ this.channels[i] = new OutputChannel(this, i, id, connectedId, getChannelType());
+ }
+ }
+
+ public OutputChannel[] channels() {
+ return this.channels;
+ }
+
+ public OutputChannel getChannel(int index) {
+ return (index < this.channels.length) ? this.channels[index] : null;
+ }
+
+ public int getNumChannels() {
+ return this.channels.length;
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+ // Shutdown
+ // -----------------------------------------------------------------------------------------------------------------
+
+ public void requestClose() throws IOException, InterruptedException {
+ for (OutputChannel channel : this.channels) {
+ channel.requestClose();
+ }
+ }
+
+ @Override
+ public boolean isClosed() {
+ if (this.closed) {
+ return true;
+ }
+
+ for (OutputChannel channel : this.channels) {
+ if (!channel.isClosed()) {
+ return false;
+ }
+ }
+
+ this.closed = true;
+ return true;
+ }
+
+ public void waitForGateToBeClosed() throws InterruptedException {
+ if (this.closed) {
+ return;
+ }
+
+ for (OutputChannel channel : this.channels) {
+ channel.waitForChannelToBeClosed();
+ }
+
+ this.closed = true;
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ @Override
+ public boolean isInputGate() {
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return "Output " + super.toString();
+ }
+
+ @Override
+ public void publishEvent(AbstractEvent event) throws IOException, InterruptedException {
+ // replaced by broadcastEvent(AbstractEvent) => TODO will be removed with changes to input side
+ }
+
+ @Override
+ public void releaseAllChannelResources() {
+ // nothing to do for buffer oriented runtime => TODO will be removed with changes to input side
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/RecordAvailabilityListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/RecordAvailabilityListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/RecordAvailabilityListener.java
new file mode 100644
index 0000000..ea1d865
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/gates/RecordAvailabilityListener.java
@@ -0,0 +1,36 @@
+/***********************************************************************************************************************
+ * 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.gates;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.runtime.io.gates.InputGate;
+
+/**
+ * This interface can be implemented by a class which shall be notified by an input gate when one of the its connected
+ * input channels has at least one record available for reading.
+ *
+ * @param <T>
+ * the type of record transported through the corresponding input gate
+ */
+public interface RecordAvailabilityListener<T extends IOReadableWritable> {
+
+ /**
+ * This method is called by an input gate when one of its connected input channels has at least one record available
+ * for reading.
+ *
+ * @param inputGate
+ * the input gate which has at least one record available
+ */
+ void reportRecordAvailability(InputGate<T> inputGate);
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ChannelManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ChannelManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ChannelManager.java
new file mode 100644
index 0000000..405d79e
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ChannelManager.java
@@ -0,0 +1,646 @@
+/***********************************************************************************************************************
+ * 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;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.nephele.execution.CancelTaskException;
+import eu.stratosphere.nephele.execution.Environment;
+import eu.stratosphere.nephele.execution.RuntimeEnvironment;
+import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
+import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
+import eu.stratosphere.runtime.io.channels.Channel;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.channels.InputChannel;
+import eu.stratosphere.runtime.io.channels.OutputChannel;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.nephele.protocols.ChannelLookupProtocol;
+import eu.stratosphere.nephele.taskmanager.Task;
+import eu.stratosphere.nephele.AbstractID;
+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.network.bufferprovider.GlobalBufferPool;
+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.InputGate;
+import eu.stratosphere.runtime.io.gates.OutputGate;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * The channel manager sets up the network buffers and dispatches data between channels.
+ */
+public final class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker {
+
+ private static final Log LOG = LogFactory.getLog(ChannelManager.class);
+
+ private final ChannelLookupProtocol channelLookupService;
+
+ private final InstanceConnectionInfo connectionInfo;
+
+ private final Map<ChannelID, Channel> channels;
+
+ private final Map<AbstractID, LocalBufferPoolOwner> localBuffersPools;
+
+ private final Map<ChannelID, EnvelopeReceiverList> receiverCache;
+
+ private final GlobalBufferPool globalBufferPool;
+
+ private final NetworkConnectionManager networkConnectionManager;
+
+ private final InetSocketAddress ourAddress;
+
+ private final SerialSingleBufferPool discardingDataPool;
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ public ChannelManager(ChannelLookupProtocol channelLookupService, InstanceConnectionInfo connectionInfo,
+ int numNetworkBuffers, int networkBufferSize) throws IOException {
+ this.channelLookupService = channelLookupService;
+ this.connectionInfo = connectionInfo;
+ this.globalBufferPool = new GlobalBufferPool(numNetworkBuffers, networkBufferSize);
+ this.networkConnectionManager = new NetworkConnectionManager(this, connectionInfo.address(), connectionInfo.dataPort());
+
+ // management data structures
+ this.channels = new ConcurrentHashMap<ChannelID, Channel>();
+ this.receiverCache = new ConcurrentHashMap<ChannelID, EnvelopeReceiverList>();
+ this.localBuffersPools = new ConcurrentHashMap<AbstractID, LocalBufferPoolOwner>();
+
+ this.ourAddress = new InetSocketAddress(connectionInfo.address(), connectionInfo.dataPort());
+
+ // a special pool if the data is to be discarded
+ this.discardingDataPool = new SerialSingleBufferPool(networkBufferSize);
+ }
+
+ public void shutdown() {
+ this.networkConnectionManager.shutDown();
+ this.globalBufferPool.destroy();
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+ // Task registration
+ // -----------------------------------------------------------------------------------------------------------------
+
+ /**
+ * Registers the given task with the channel manager.
+ *
+ * @param task the task to be registered
+ * @throws InsufficientResourcesException thrown if not enough buffers available to safely run this task
+ */
+ public void register(Task task) throws InsufficientResourcesException {
+ // Check if we can safely run this task with the given buffers
+ ensureBufferAvailability(task);
+
+ RuntimeEnvironment environment = task.getRuntimeEnvironment();
+
+ // -------------------------------------------------------------------------------------------------------------
+ // Register output channels
+ // -------------------------------------------------------------------------------------------------------------
+
+ environment.registerGlobalBufferPool(this.globalBufferPool);
+
+ if (this.localBuffersPools.containsKey(task.getVertexID())) {
+ throw new IllegalStateException("Vertex " + task.getVertexID() + " has a previous buffer pool owner");
+ }
+
+ for (OutputGate gate : environment.outputGates()) {
+ // add receiver list hints
+ for (OutputChannel channel : gate.channels()) {
+ // register envelope dispatcher with the channel
+ channel.registerEnvelopeDispatcher(this);
+
+ switch (channel.getChannelType()) {
+ case IN_MEMORY:
+ addReceiverListHint(channel.getID(), channel.getConnectedId());
+ break;
+ case NETWORK:
+ addReceiverListHint(channel.getConnectedId(), channel.getID());
+ break;
+ }
+
+ this.channels.put(channel.getID(), channel);
+ }
+ }
+
+ this.localBuffersPools.put(task.getVertexID(), environment);
+
+ // -------------------------------------------------------------------------------------------------------------
+ // Register input channels
+ // -------------------------------------------------------------------------------------------------------------
+
+ // register global
+ for (InputGate<?> gate : environment.inputGates()) {
+ gate.registerGlobalBufferPool(this.globalBufferPool);
+
+ for (int i = 0; i < gate.getNumberOfInputChannels(); i++) {
+ InputChannel<? extends IOReadableWritable> channel = gate.getInputChannel(i);
+ channel.registerEnvelopeDispatcher(this);
+
+ if (channel.getChannelType() == ChannelType.IN_MEMORY) {
+ addReceiverListHint(channel.getID(), channel.getConnectedId());
+ }
+
+ this.channels.put(channel.getID(), channel);
+ }
+
+ this.localBuffersPools.put(gate.getGateID(), gate);
+ }
+
+ // the number of channels per buffers has changed after unregistering the task
+ // => redistribute the number of designated buffers of the registered local buffer pools
+ redistributeBuffers();
+ }
+
+ /**
+ * Unregisters the given task from the channel manager.
+ *
+ * @param vertexId the ID of the task to be unregistered
+ * @param task the task to be unregistered
+ */
+ public void unregister(ExecutionVertexID vertexId, Task task) {
+ final Environment environment = task.getEnvironment();
+
+ // destroy and remove OUTPUT channels from registered channels and cache
+ for (ChannelID id : environment.getOutputChannelIDs()) {
+ Channel channel = this.channels.remove(id);
+ if (channel != null) {
+ channel.destroy();
+ }
+
+ this.receiverCache.remove(channel);
+ }
+
+ // destroy and remove INPUT channels from registered channels and cache
+ for (ChannelID id : environment.getInputChannelIDs()) {
+ Channel channel = this.channels.remove(id);
+ if (channel != null) {
+ channel.destroy();
+ }
+
+ this.receiverCache.remove(channel);
+ }
+
+ // clear and remove INPUT side buffer pools
+ for (GateID id : environment.getInputGateIDs()) {
+ LocalBufferPoolOwner bufferPool = this.localBuffersPools.remove(id);
+ if (bufferPool != null) {
+ bufferPool.clearLocalBufferPool();
+ }
+ }
+
+ // clear and remove OUTPUT side buffer pool
+ LocalBufferPoolOwner bufferPool = this.localBuffersPools.remove(vertexId);
+ if (bufferPool != null) {
+ bufferPool.clearLocalBufferPool();
+ }
+
+ // the number of channels per buffers has changed after unregistering the task
+ // => redistribute the number of designated buffers of the registered local buffer pools
+ redistributeBuffers();
+ }
+
+ /**
+ * Ensures that the channel manager has enough buffers to execute the given task.
+ * <p>
+ * If there is less than one buffer per channel available, an InsufficientResourcesException will be thrown,
+ * because of possible deadlocks. With more then one buffer per channel, deadlock-freedom is guaranteed.
+ *
+ * @param task task to be executed
+ * @throws InsufficientResourcesException thrown if not enough buffers available to execute the task
+ */
+ private void ensureBufferAvailability(Task task) throws InsufficientResourcesException {
+ Environment env = task.getEnvironment();
+
+ int numBuffers = this.globalBufferPool.numBuffers();
+ // existing channels + channels of the task
+ int numChannels = this.channels.size() + env.getNumberOfOutputChannels() + env.getNumberOfInputChannels();
+
+ // need at least one buffer per channel
+ if (numBuffers / numChannels < 1) {
+ String msg = String.format("%s has not enough buffers to safely execute %s (%d buffers missing)",
+ this.connectionInfo.hostname(), env.getTaskName(), numChannels - numBuffers);
+
+ throw new InsufficientResourcesException(msg);
+ }
+ }
+
+ /**
+ * Redistributes the buffers among the registered buffer pools. This method is called after each task registration
+ * and unregistration.
+ * <p>
+ * Every registered buffer pool gets buffers according to its number of channels weighted by the current buffer to
+ * channel ratio.
+ */
+ private void redistributeBuffers() {
+ if (this.localBuffersPools.isEmpty() | this.channels.size() == 0) {
+ return;
+ }
+
+ int numBuffers = this.globalBufferPool.numBuffers();
+ int numChannels = this.channels.size();
+
+ double buffersPerChannel = numBuffers / (double) numChannels;
+
+ if (buffersPerChannel < 1.0) {
+ throw new RuntimeException("System has not enough buffers to execute tasks.");
+ }
+
+ // redistribute number of designated buffers per buffer pool
+ for (LocalBufferPoolOwner bufferPool : this.localBuffersPools.values()) {
+ int numDesignatedBuffers = (int) Math.ceil(buffersPerChannel * bufferPool.getNumberOfChannels());
+ bufferPool.setDesignatedNumberOfBuffers(numDesignatedBuffers);
+ }
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+ // Envelope processing
+ // -----------------------------------------------------------------------------------------------------------------
+
+ private void releaseEnvelope(Envelope envelope) {
+ Buffer buffer = envelope.getBuffer();
+ if (buffer != null) {
+ buffer.recycleBuffer();
+ }
+ }
+
+ private void addReceiverListHint(ChannelID source, ChannelID localReceiver) {
+ EnvelopeReceiverList receiverList = new EnvelopeReceiverList(localReceiver);
+
+ if (this.receiverCache.put(source, receiverList) != null) {
+ LOG.warn("Receiver cache already contained entry for " + source);
+ }
+ }
+
+ private void addReceiverListHint(ChannelID source, RemoteReceiver remoteReceiver) {
+ EnvelopeReceiverList receiverList = new EnvelopeReceiverList(remoteReceiver);
+
+ if (this.receiverCache.put(source, receiverList) != null) {
+ LOG.warn("Receiver cache already contained entry for " + source);
+ }
+ }
+
+ private void generateSenderHint(Envelope envelope, RemoteReceiver receiver) {
+ Channel channel = this.channels.get(envelope.getSource());
+ if (channel == null) {
+ LOG.error("Cannot find channel for channel ID " + envelope.getSource());
+ return;
+ }
+
+ // Only generate sender hints for output channels
+ if (channel.isInputChannel()) {
+ return;
+ }
+
+ final ChannelID targetChannelID = channel.getConnectedId();
+ final int connectionIndex = receiver.getConnectionIndex();
+
+ final RemoteReceiver ourAddress = new RemoteReceiver(this.ourAddress, connectionIndex);
+ final Envelope senderHint = SenderHintEvent.createEnvelopeWithEvent(envelope, targetChannelID, ourAddress);
+
+ this.networkConnectionManager.queueEnvelopeForTransfer(receiver, senderHint);
+ }
+
+ /**
+ * Returns the list of receivers for transfer envelopes produced by the channel with the given source channel ID.
+ *
+ * @param jobID
+ * the ID of the job the given channel ID belongs to
+ * @param sourceChannelID
+ * 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
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ private EnvelopeReceiverList getReceiverList(JobID jobID, ChannelID sourceChannelID, boolean reportException) throws IOException {
+ EnvelopeReceiverList receiverList = this.receiverCache.get(sourceChannelID);
+
+ if (receiverList != null) {
+ return receiverList;
+ }
+
+ while (true) {
+ ConnectionInfoLookupResponse lookupResponse;
+ synchronized (this.channelLookupService) {
+ lookupResponse = this.channelLookupService.lookupConnectionInfo(this.connectionInfo, jobID, sourceChannelID);
+ }
+
+ if (lookupResponse.receiverReady()) {
+ receiverList = new EnvelopeReceiverList(lookupResponse);
+ break;
+ }
+ else if (lookupResponse.receiverNotReady()) {
+ try {
+ Thread.sleep(500);
+ } catch (InterruptedException e) {
+ if (reportException) {
+ throw new IOException("Lookup was interrupted.");
+ } else {
+ return null;
+ }
+ }
+ }
+ else if (lookupResponse.isJobAborting()) {
+ if (reportException) {
+ throw new CancelTaskException();
+ } else {
+ return null;
+ }
+ }
+ else if (lookupResponse.receiverNotFound()) {
+ if (reportException) {
+ throw new IOException("Could not find the receiver for Job " + jobID + ", channel with source id " + sourceChannelID);
+ } else {
+ return null;
+ }
+ }
+ else {
+ throw new IllegalStateException("Unrecognized response to channel lookup.");
+ }
+ }
+
+ this.receiverCache.put(sourceChannelID, receiverList);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Receivers for source channel ID " + sourceChannelID + " at task manager " + this.connectionInfo +
+ ": " + receiverList);
+ }
+
+ return receiverList;
+ }
+
+ /**
+ * Invalidates the entries identified by the given channel IDs from the receiver lookup cache.
+ *
+ * @param channelIDs channel IDs for entries to invalidate
+ */
+ public void invalidateLookupCacheEntries(Set<ChannelID> channelIDs) {
+ for (ChannelID id : channelIDs) {
+ this.receiverCache.remove(id);
+ }
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+ // EnvelopeDispatcher methods
+ // -----------------------------------------------------------------------------------------------------------------
+
+ @Override
+ public void dispatchFromOutputChannel(Envelope envelope) throws IOException, InterruptedException {
+ EnvelopeReceiverList receiverList = getReceiverListForEnvelope(envelope, true);
+
+ Buffer srcBuffer = envelope.getBuffer();
+ Buffer destBuffer = null;
+
+ boolean success = false;
+
+ try {
+ if (receiverList.hasLocalReceiver()) {
+ ChannelID receiver = receiverList.getLocalReceiver();
+ Channel channel = this.channels.get(receiver);
+
+ if (channel == null) {
+ throw new LocalReceiverCancelledException(receiver);
+ }
+
+ if (!channel.isInputChannel()) {
+ throw new IOException("Local receiver " + receiver + " is not an input channel.");
+ }
+
+ InputChannel<?> inputChannel = (InputChannel<?>) channel;
+
+ // copy the buffer into the memory space of the receiver
+ if (srcBuffer != null) {
+ try {
+ destBuffer = inputChannel.requestBufferBlocking(srcBuffer.size());
+ } catch (InterruptedException e) {
+ throw new IOException(e.getMessage());
+ }
+
+ srcBuffer.copyToBuffer(destBuffer);
+ envelope.setBuffer(destBuffer);
+ srcBuffer.recycleBuffer();
+ }
+
+ inputChannel.queueEnvelope(envelope);
+ success = true;
+ }
+ else if (receiverList.hasRemoteReceiver()) {
+ RemoteReceiver remoteReceiver = receiverList.getRemoteReceiver();
+
+ // Generate sender hint before sending the first envelope over the network
+ if (envelope.getSequenceNumber() == 0) {
+ generateSenderHint(envelope, remoteReceiver);
+ }
+
+ this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, envelope);
+ success = true;
+ }
+ } finally {
+ if (!success) {
+ if (srcBuffer != null) {
+ srcBuffer.recycleBuffer();
+ }
+ if (destBuffer != null) {
+ destBuffer.recycleBuffer();
+ }
+ }
+ }
+ }
+
+ @Override
+ public void dispatchFromInputChannel(Envelope envelope) throws IOException, InterruptedException {
+ // this method sends only events back from input channels to output channels
+ // sanity check that we have no buffer
+ if (envelope.getBuffer() != null) {
+ throw new RuntimeException("Error: This method can only process envelopes without buffers.");
+ }
+
+ EnvelopeReceiverList receiverList = getReceiverListForEnvelope(envelope, true);
+
+ if (receiverList.hasLocalReceiver()) {
+ ChannelID receiver = receiverList.getLocalReceiver();
+ Channel channel = this.channels.get(receiver);
+
+ if (channel == null) {
+ throw new LocalReceiverCancelledException(receiver);
+ }
+
+ if (channel.isInputChannel()) {
+ throw new IOException("Local receiver " + receiver + " of backward event is not an output channel.");
+ }
+
+ OutputChannel outputChannel = (OutputChannel) channel;
+ outputChannel.queueEnvelope(envelope);
+ }
+ else if (receiverList.hasRemoteReceiver()) {
+ RemoteReceiver remoteReceiver = receiverList.getRemoteReceiver();
+
+ // Generate sender hint before sending the first envelope over the network
+ if (envelope.getSequenceNumber() == 0) {
+ generateSenderHint(envelope, remoteReceiver);
+ }
+
+ this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, envelope);
+ }
+ }
+
+ /**
+ *
+ */
+ @Override
+ public void dispatchFromNetwork(Envelope envelope) throws IOException, InterruptedException {
+ // ========================================================================================
+ // IMPORTANT
+ //
+ // This method is called by the network I/O thread that reads the incoming TCP
+ // connections. This method must have minimal overhead and not throw exception if
+ // something is wrong with a job or individual transmission, but only when something
+ // is fundamentally broken in the system.
+ // ========================================================================================
+
+ // the sender hint event is to let the receiver know where exactly the envelope came from.
+ // the receiver will cache the sender id and its connection info in its local lookup table
+ // that allows the receiver to send envelopes to the sender without first pinging the job manager
+ // for the sender's connection info
+
+ // Check if the envelope is the special envelope with the sender hint event
+ if (SenderHintEvent.isSenderHintEvent(envelope)) {
+ // Check if this is the final destination of the sender hint event before adding it
+ final SenderHintEvent seh = (SenderHintEvent) envelope.deserializeEvents().get(0);
+ if (this.channels.get(seh.getSource()) != null) {
+ addReceiverListHint(seh.getSource(), seh.getRemoteReceiver());
+ return;
+ }
+ }
+
+ // try and get the receiver list. if we cannot get it anymore, the task has been cleared
+ // the code frees the envelope on exception, so we need not to anything
+ EnvelopeReceiverList receiverList = getReceiverListForEnvelope(envelope, false);
+ if (receiverList == null) {
+ // receiver is cancelled and cleaned away
+ releaseEnvelope(envelope);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Dropping envelope for cleaned up receiver.");
+ }
+
+ return;
+ }
+
+ if (!receiverList.hasLocalReceiver() || receiverList.hasRemoteReceiver()) {
+ throw new IOException("Bug in network stack: Envelope dispatched from the incoming network pipe has no local receiver or has a remote receiver");
+ }
+
+ ChannelID localReceiver = receiverList.getLocalReceiver();
+ Channel channel = this.channels.get(localReceiver);
+
+ // if the channel is null, it means that receiver has been cleared already (cancelled or failed).
+ // release the buffer immediately
+ if (channel == null) {
+ releaseEnvelope(envelope);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Dropping envelope for cancelled receiver " + localReceiver);
+ }
+ }
+ else {
+ channel.queueEnvelope(envelope);
+ }
+ }
+
+ /**
+ *
+ * Upon an exception, this method frees the envelope.
+ *
+ * @param envelope
+ * @return
+ * @throws IOException
+ */
+ private final EnvelopeReceiverList getReceiverListForEnvelope(Envelope envelope, boolean reportException) throws IOException {
+ try {
+ return getReceiverList(envelope.getJobID(), envelope.getSource(), reportException);
+ } catch (IOException e) {
+ releaseEnvelope(envelope);
+ throw e;
+ } catch (CancelTaskException e) {
+ releaseEnvelope(envelope);
+ throw e;
+ }
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+ // BufferProviderBroker methods
+ // -----------------------------------------------------------------------------------------------------------------
+
+ @Override
+ public BufferProvider getBufferProvider(JobID jobID, ChannelID sourceChannelID) throws IOException {
+ EnvelopeReceiverList receiverList = getReceiverList(jobID, sourceChannelID, false);
+
+ // check if the receiver is already gone
+ if (receiverList == null) {
+ return this.discardingDataPool;
+ }
+
+ if (!receiverList.hasLocalReceiver() || receiverList.hasRemoteReceiver()) {
+ throw new IOException("The destination to be looked up is not a single local endpoint.");
+ }
+
+
+ ChannelID localReceiver = receiverList.getLocalReceiver();
+ Channel channel = this.channels.get(localReceiver);
+
+ if (channel == null) {
+ // receiver is already canceled
+ return this.discardingDataPool;
+ }
+
+ if (!channel.isInputChannel()) {
+ throw new IOException("Channel context for local receiver " + localReceiver + " is not an input channel context");
+ }
+
+ return (InputChannel<?>) channel;
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ public void logBufferUtilization() {
+ System.out.println("Buffer utilization at " + System.currentTimeMillis());
+
+ System.out.println("\tUnused global buffers: " + this.globalBufferPool.numAvailableBuffers());
+
+ System.out.println("\tLocal buffer pool status:");
+
+ for (LocalBufferPoolOwner bufferPool : this.localBuffersPools.values()) {
+ bufferPool.logBufferUtilization();
+ }
+
+ this.networkConnectionManager.logBufferUtilization();
+
+ System.out.println("\tIncoming connections:");
+
+ for (Channel channel : this.channels.values()) {
+ if (channel.isInputChannel()) {
+ ((InputChannel<?>) channel).logQueuedEnvelopes();
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ConnectionInfoLookupResponse.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ConnectionInfoLookupResponse.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ConnectionInfoLookupResponse.java
new file mode 100644
index 0000000..aeb5025
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ConnectionInfoLookupResponse.java
@@ -0,0 +1,143 @@
+/***********************************************************************************************************************
+ *
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+
+public class ConnectionInfoLookupResponse implements IOReadableWritable {
+
+ private enum ReturnCode {
+ NOT_FOUND, FOUND_AND_RECEIVER_READY, FOUND_BUT_RECEIVER_NOT_READY, JOB_IS_ABORTING
+ };
+
+ // was request successful?
+ private ReturnCode returnCode;
+
+ private RemoteReceiver remoteTarget;
+
+ private ChannelID localTarget;
+
+
+ public ConnectionInfoLookupResponse() {}
+
+ public ConnectionInfoLookupResponse(ReturnCode code) {
+ this.returnCode = code;
+ this.remoteTarget = null;
+ this.localTarget = null;
+ }
+
+ public ConnectionInfoLookupResponse(ReturnCode code, ChannelID localTarget) {
+ this.returnCode = code;
+ this.remoteTarget = null;
+ this.localTarget = localTarget;
+ }
+
+ public ConnectionInfoLookupResponse(ReturnCode code, RemoteReceiver receiver) {
+ this.returnCode = code;
+ this.remoteTarget = receiver;
+ this.localTarget = null;
+ }
+
+ public RemoteReceiver getRemoteTarget() {
+ return this.remoteTarget;
+ }
+
+ public ChannelID getLocalTarget() {
+ return this.localTarget;
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ this.returnCode = ReturnCode.values()[in.readInt()];
+
+ if (in.readBoolean()) {
+ this.remoteTarget = new RemoteReceiver();
+ this.remoteTarget.read(in);
+ }
+ if (in.readBoolean()) {
+ this.localTarget = new ChannelID();
+ this.localTarget.read(in);
+ }
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeInt(this.returnCode.ordinal());
+
+ if (this.remoteTarget != null) {
+ out.writeBoolean(true);
+ this.remoteTarget.write(out);
+ } else {
+ out.writeBoolean(false);
+ }
+
+ if (this.localTarget != null) {
+ out.writeBoolean(true);
+ this.localTarget.write(out);
+ } else {
+ out.writeBoolean(false);
+ }
+ }
+
+ public boolean receiverNotFound() {
+ return (this.returnCode == ReturnCode.NOT_FOUND);
+ }
+
+ public boolean receiverNotReady() {
+ return (this.returnCode == ReturnCode.FOUND_BUT_RECEIVER_NOT_READY);
+ }
+
+ public boolean receiverReady() {
+ return (this.returnCode == ReturnCode.FOUND_AND_RECEIVER_READY);
+ }
+
+ public boolean isJobAborting() {
+ return (this.returnCode == ReturnCode.JOB_IS_ABORTING);
+ }
+
+
+ public static ConnectionInfoLookupResponse createReceiverFoundAndReady(ChannelID targetChannelID) {
+ return new ConnectionInfoLookupResponse(ReturnCode.FOUND_AND_RECEIVER_READY, targetChannelID);
+ }
+
+ public static ConnectionInfoLookupResponse createReceiverFoundAndReady(RemoteReceiver remoteReceiver) {
+ return new ConnectionInfoLookupResponse(ReturnCode.FOUND_AND_RECEIVER_READY, remoteReceiver);
+ }
+
+ public static ConnectionInfoLookupResponse createReceiverNotFound() {
+ return new ConnectionInfoLookupResponse(ReturnCode.NOT_FOUND);
+ }
+
+ public static ConnectionInfoLookupResponse createReceiverNotReady() {
+ return new ConnectionInfoLookupResponse(ReturnCode.FOUND_BUT_RECEIVER_NOT_READY);
+ }
+
+ public static ConnectionInfoLookupResponse createJobIsAborting() {
+ return new ConnectionInfoLookupResponse(ReturnCode.JOB_IS_ABORTING);
+ }
+
+
+ @Override
+ public String toString() {
+ return this.returnCode.name() + ", local target: " + this.localTarget + ", remoteTarget: " + this.remoteTarget;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/InsufficientResourcesException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/InsufficientResourcesException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/InsufficientResourcesException.java
new file mode 100644
index 0000000..0d2fcd4
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/InsufficientResourcesException.java
@@ -0,0 +1,37 @@
+/***********************************************************************************************************************
+ * 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;
+
+/**
+ * This exception is thrown by the {@link ChannelManager} to indicate that a task cannot be accepted because
+ * there are not enough resources available to safely execute it.
+ *
+ */
+public final class InsufficientResourcesException extends Exception {
+
+ /**
+ * The generated serial version UID.
+ */
+ private static final long serialVersionUID = -8977049569413215169L;
+
+ /**
+ * Constructs a new insufficient resources exception.
+ *
+ * @param msg
+ * the message describing the exception
+ */
+ InsufficientResourcesException(final String msg) {
+ super(msg);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/LocalReceiverCancelledException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/LocalReceiverCancelledException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/LocalReceiverCancelledException.java
new file mode 100644
index 0000000..769ada5
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/LocalReceiverCancelledException.java
@@ -0,0 +1,37 @@
+/***********************************************************************************************************************
+ *
+ * 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.nephele.execution.CancelTaskException;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+
+
+/**
+ *
+ */
+public class LocalReceiverCancelledException extends CancelTaskException {
+ private static final long serialVersionUID = 1L;
+
+ private final ChannelID receiver;
+
+ public LocalReceiverCancelledException(ChannelID receiver) {
+ this.receiver = receiver;
+ }
+
+
+ public ChannelID getReceiver() {
+ return receiver;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java
new file mode 100644
index 0000000..44ec642
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/NetworkConnectionManager.java
@@ -0,0 +1,176 @@
+/***********************************************************************************************************************
+ * 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());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/RemoteReceiver.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/RemoteReceiver.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/RemoteReceiver.java
new file mode 100644
index 0000000..da36ad0
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/RemoteReceiver.java
@@ -0,0 +1,157 @@
+/***********************************************************************************************************************
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.util.StringUtils;
+
+/**
+ * Objects of this class uniquely identify a connection to a remote {@link TaskManager}.
+ *
+ */
+public final class RemoteReceiver implements IOReadableWritable {
+
+ /**
+ * The address of the connection to the remote {@link TaskManager}.
+ */
+ private InetSocketAddress connectionAddress;
+
+ /**
+ * The index of the connection to the remote {@link TaskManager}.
+ */
+ private int connectionIndex;
+
+ /**
+ * Constructs a new remote receiver object.
+ *
+ * @param connectionAddress
+ * the address of the connection to the remote {@link TaskManager}
+ * @param connectionIndex
+ * the index of the connection to the remote {@link TaskManager}
+ */
+ public RemoteReceiver(final InetSocketAddress connectionAddress, final int connectionIndex) {
+
+ if (connectionAddress == null) {
+ throw new IllegalArgumentException("Argument connectionAddress must not be null");
+ }
+
+ if (connectionIndex < 0) {
+ throw new IllegalArgumentException("Argument connectionIndex must be a non-negative integer number");
+ }
+
+ this.connectionAddress = connectionAddress;
+ this.connectionIndex = connectionIndex;
+ }
+
+ /**
+ * Default constructor for serialization/deserialization.
+ */
+ public RemoteReceiver() {
+ this.connectionAddress = null;
+ this.connectionIndex = -1;
+ }
+
+ /**
+ * Returns the address of the connection to the remote {@link TaskManager}.
+ *
+ * @return the address of the connection to the remote {@link TaskManager}
+ */
+ public InetSocketAddress getConnectionAddress() {
+
+ return this.connectionAddress;
+ }
+
+ /**
+ * Returns the index of the connection to the remote {@link TaskManager}.
+ *
+ * @return the index of the connection to the remote {@link TaskManager}
+ */
+ public int getConnectionIndex() {
+
+ return this.connectionIndex;
+ }
+
+
+ @Override
+ public int hashCode() {
+
+ return this.connectionAddress.hashCode() + (31 * this.connectionIndex);
+ }
+
+
+ @Override
+ public boolean equals(final Object obj) {
+
+ if (!(obj instanceof RemoteReceiver)) {
+ return false;
+ }
+
+ final RemoteReceiver rr = (RemoteReceiver) obj;
+ if (!this.connectionAddress.equals(rr.connectionAddress)) {
+ return false;
+ }
+
+ if (this.connectionIndex != rr.connectionIndex) {
+ return false;
+ }
+
+ return true;
+ }
+
+
+ @Override
+ public void write(final DataOutput out) throws IOException {
+
+ final InetAddress ia = this.connectionAddress.getAddress();
+ out.writeInt(ia.getAddress().length);
+ out.write(ia.getAddress());
+ out.writeInt(this.connectionAddress.getPort());
+
+ out.writeInt(this.connectionIndex);
+ }
+
+
+ @Override
+ public void read(final DataInput in) throws IOException {
+
+ final int addr_length = in.readInt();
+ final byte[] address = new byte[addr_length];
+ in.readFully(address);
+
+ InetAddress ia = null;
+ try {
+ ia = InetAddress.getByAddress(address);
+ } catch (UnknownHostException uhe) {
+ throw new IOException(StringUtils.stringifyException(uhe));
+ }
+ final int port = in.readInt();
+ this.connectionAddress = new InetSocketAddress(ia, port);
+
+ this.connectionIndex = in.readInt();
+ }
+
+
+ @Override
+ public String toString() {
+
+ return this.connectionAddress + " (" + this.connectionIndex + ")";
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java
new file mode 100644
index 0000000..32be058
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java
@@ -0,0 +1,117 @@
+/***********************************************************************************************************************
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import eu.stratosphere.nephele.event.task.AbstractEvent;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.network.envelope.Envelope;
+
+public final class SenderHintEvent extends AbstractEvent {
+
+ /**
+ * The sequence number that will be set for transfer envelopes which contain the sender hint event.
+ */
+ private static final int SENDER_HINT_SEQUENCE_NUMBER = 0;
+
+ private final ChannelID source;
+
+ private final RemoteReceiver remoteReceiver;
+
+ SenderHintEvent(final ChannelID source, final RemoteReceiver remoteReceiver) {
+
+ if (source == null) {
+ throw new IllegalArgumentException("Argument source must not be null");
+ }
+
+ if (remoteReceiver == null) {
+ throw new IllegalArgumentException("Argument remoteReceiver must not be null");
+ }
+
+ this.source = source;
+ this.remoteReceiver = remoteReceiver;
+ }
+
+ public SenderHintEvent() {
+
+ this.source = new ChannelID();
+ this.remoteReceiver = new RemoteReceiver();
+ }
+
+ public ChannelID getSource() {
+
+ return this.source;
+ }
+
+ public RemoteReceiver getRemoteReceiver() {
+
+ return this.remoteReceiver;
+ }
+
+
+ @Override
+ public void write(final DataOutput out) throws IOException {
+
+ this.source.write(out);
+ this.remoteReceiver.write(out);
+ }
+
+
+ @Override
+ public void read(final DataInput in) throws IOException {
+
+ this.source.read(in);
+ this.remoteReceiver.read(in);
+ }
+
+ public static Envelope createEnvelopeWithEvent(final Envelope originalEnvelope,
+ final ChannelID source, final RemoteReceiver remoteReceiver) {
+
+ final Envelope envelope = new Envelope(SENDER_HINT_SEQUENCE_NUMBER,
+ originalEnvelope.getJobID(), originalEnvelope.getSource());
+
+ final SenderHintEvent senderEvent = new SenderHintEvent(source, remoteReceiver);
+ envelope.serializeEventList(Arrays.asList(senderEvent));
+
+ return envelope;
+ }
+
+ static boolean isSenderHintEvent(final Envelope envelope) {
+
+ if (envelope.getSequenceNumber() != SENDER_HINT_SEQUENCE_NUMBER) {
+ return false;
+ }
+
+ if (envelope.getBuffer() != null) {
+ return false;
+ }
+
+ List<? extends AbstractEvent> events = envelope.deserializeEvents();
+
+ if (events.size() != 1) {
+ return false;
+ }
+
+ if (!(events.get(0) instanceof SenderHintEvent)) {
+ return false;
+ }
+
+ return true;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java
new file mode 100644
index 0000000..1d23e93
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferAvailabilityListener.java
@@ -0,0 +1,28 @@
+/***********************************************************************************************************************
+ * 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.bufferprovider;
+
+/**
+ * This interface must be implemented to receive a notification from a {@link BufferProvider} when an empty
+ * {@link eu.stratosphere.runtime.io.Buffer} has
+ * become available again.
+ *
+ */
+public interface BufferAvailabilityListener {
+
+ /**
+ * Indicates that at least one {@link eu.stratosphere.runtime.io.Buffer} has become available again.
+ */
+ void bufferAvailable();
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
new file mode 100644
index 0000000..e3085ee
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
@@ -0,0 +1,69 @@
+/***********************************************************************************************************************
+ * 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.bufferprovider;
+
+import eu.stratosphere.runtime.io.Buffer;
+
+import java.io.IOException;
+
+public interface BufferProvider {
+
+ /**
+ * Requests a buffer with a minimum size of <code>minBufferSize</code>. The method returns immediately, even if the
+ * request could not be fulfilled.
+ *
+ * @param minBufferSize minimum size of the requested buffer (in bytes)
+ * @return buffer with at least the requested size or <code>null</code> if no such buffer is currently available
+ * @throws IOException
+ */
+ Buffer requestBuffer(int minBufferSize) throws IOException;
+
+ /**
+ * Requests a buffer with a minimum size of <code>minBufferSize</code>. The method blocks until the request has
+ * been fulfilled or {@link #reportAsynchronousEvent()} has been called.
+ *
+ * @param minBufferSize minimum size of the requested buffer (in bytes)
+ * @return buffer with at least the requested size
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ Buffer requestBufferBlocking(int minBufferSize) throws IOException, InterruptedException;
+
+ /**
+ * Returns the size of buffers (in bytes) available at this buffer provider.
+ *
+ * @return size of buffers (in bytes) available at this buffer provider
+ */
+ int getBufferSize();
+
+ /**
+ * Reports an asynchronous event and interrupts each blocking method of this buffer provider in order to allow the
+ * blocked thread to respond to the event.
+ */
+ void reportAsynchronousEvent();
+
+ /**
+ * Registers the given {@link BufferAvailabilityListener} with an empty buffer pool.
+ * <p>
+ * The registration only succeeds, if the buffer pool is empty and has not been destroyed yet.
+ * <p>
+ * The registered listener will receive a notification when at least one buffer has become available again. After
+ * the notification, the listener will be unregistered.
+ *
+ * @param listener the listener to be registered
+ * @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
+ */
+ boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener);
+}
[22/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java
deleted file mode 100644
index 71cad15..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeInputGate.java
+++ /dev/null
@@ -1,330 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.io.channels.AbstractInputChannel;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryInputChannel;
-import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkInputChannel;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * In Nephele input gates are a specialization of general gates and connect input channels and record readers. As
- * channels, input gates are always parameterized to a specific type of record which they can transport. In contrast to
- * output gates input gates can be associated with a {@link DistributionPattern} object which dictates the concrete
- * wiring between two groups of vertices.
- *
- * @param <T> The type of record that can be transported through this gate.
- */
-public class RuntimeInputGate<T extends IOReadableWritable> extends AbstractGate<T> implements InputGate<T> {
-
- /**
- * The log object used for debugging.
- */
- private static final Log LOG = LogFactory.getLog(InputGate.class);
-
- /**
- * The deserializer factory used to instantiate the deserializers that construct records from byte streams.
- */
- private final RecordDeserializerFactory<T> deserializerFactory;
-
- /**
- * The list of input channels attached to this input gate.
- */
- private final ArrayList<AbstractInputChannel<T>> inputChannels = new ArrayList<AbstractInputChannel<T>>();
-
- /**
- * Queue with indices of channels that store at least one available record.
- */
- private final BlockingQueue<Integer> availableChannels = new LinkedBlockingQueue<Integer>();
-
- /**
- * The listener object to be notified when a channel has at least one record available.
- */
- private final AtomicReference<RecordAvailabilityListener<T>> recordAvailabilityListener = new AtomicReference<RecordAvailabilityListener<T>>(null);
-
-
- private AbstractTaskEvent currentEvent;
-
- /**
- * If the value of this variable is set to <code>true</code>, the input gate is closed.
- */
- private boolean isClosed = false;
-
- /**
- * The channel to read from next.
- */
- private int channelToReadFrom = -1;
-
- /**
- * Constructs a new runtime input gate.
- *
- * @param jobID
- * the ID of the job this input gate belongs to
- * @param gateID
- * the ID of the gate
- * @param deserializerFactory
- * The factory used to instantiate the deserializers that construct records from byte streams.
- * @param index
- * the index assigned to this input gate at the {@link Environment} object
- */
- public RuntimeInputGate(final JobID jobID, final GateID gateID,
- final RecordDeserializerFactory<T> deserializerFactory, final int index) {
- super(jobID, gateID, index);
- this.deserializerFactory = deserializerFactory;
- }
-
- /**
- * Adds a new input channel to the input gate.
- *
- * @param inputChannel
- * the input channel to be added.
- */
- private void addInputChannel(AbstractInputChannel<T> inputChannel) {
- // in high DOPs, this can be a serious performance issue, as adding all channels and checking linearly has a
- // quadratic complexity!
- if (!this.inputChannels.contains(inputChannel)) {
- this.inputChannels.add(inputChannel);
- }
- }
-
- /**
- * Removes the input channel with the given ID from the input gate if it exists.
- *
- * @param inputChannelID
- * the ID of the channel to be removed
- */
- public void removeInputChannel(ChannelID inputChannelID) {
-
- for (int i = 0; i < this.inputChannels.size(); i++) {
-
- final AbstractInputChannel<T> inputChannel = this.inputChannels.get(i);
- if (inputChannel.getID().equals(inputChannelID)) {
- this.inputChannels.remove(i);
- return;
- }
- }
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Cannot find output channel with ID " + inputChannelID + " to remove");
- }
- }
-
- @Override
- public boolean isInputGate() {
- return true;
- }
-
- @Override
- public int getNumberOfInputChannels() {
- return this.inputChannels.size();
- }
-
- @Override
- public AbstractInputChannel<T> getInputChannel(int pos) {
- return this.inputChannels.get(pos);
- }
-
-
- @Override
- public NetworkInputChannel<T> createNetworkInputChannel(final InputGate<T> inputGate, final ChannelID channelID,
- final ChannelID connectedChannelID) {
-
- final NetworkInputChannel<T> enic = new NetworkInputChannel<T>(inputGate, this.inputChannels.size(),
- this.deserializerFactory.createDeserializer(), channelID, connectedChannelID);
- addInputChannel(enic);
-
- return enic;
- }
-
-
- @Override
- public InMemoryInputChannel<T> createInMemoryInputChannel(final InputGate<T> inputGate, final ChannelID channelID,
- final ChannelID connectedChannelID) {
-
- final InMemoryInputChannel<T> eimic = new InMemoryInputChannel<T>(inputGate, this.inputChannels.size(),
- this.deserializerFactory.createDeserializer(), channelID, connectedChannelID);
- addInputChannel(eimic);
-
- return eimic;
- }
-
-
- @Override
- public InputChannelResult readRecord(T target) throws IOException, InterruptedException {
-
- if (this.channelToReadFrom == -1) {
- if (this.isClosed()) {
- return InputChannelResult.END_OF_STREAM;
- }
-
- if (Thread.interrupted()) {
- throw new InterruptedException();
- }
-
- this.channelToReadFrom = waitForAnyChannelToBecomeAvailable();
- }
-
- InputChannelResult result = this.getInputChannel(this.channelToReadFrom).readRecord(target);
- switch (result) {
- case INTERMEDIATE_RECORD_FROM_BUFFER: // full record and we can stay on the same channel
- return InputChannelResult.INTERMEDIATE_RECORD_FROM_BUFFER;
-
- case LAST_RECORD_FROM_BUFFER: // full record, but we must switch the channel afterwards
- this.channelToReadFrom = -1;
- return InputChannelResult.LAST_RECORD_FROM_BUFFER;
-
- case END_OF_SUPERSTEP:
- this.channelToReadFrom = -1;
- return InputChannelResult.END_OF_SUPERSTEP;
-
- case TASK_EVENT: // task event
- this.currentEvent = this.getInputChannel(this.channelToReadFrom).getCurrentEvent();
- this.channelToReadFrom = -1; // event always marks a unit as consumed
- return InputChannelResult.TASK_EVENT;
-
- case NONE: // internal event or an incomplete record that needs further chunks
- // the current unit is exhausted
- this.channelToReadFrom = -1;
- return InputChannelResult.NONE;
-
- case END_OF_STREAM: // channel is done
- this.channelToReadFrom = -1;
- return isClosed() ? InputChannelResult.END_OF_STREAM : InputChannelResult.NONE;
-
- default: // silence the compiler
- throw new RuntimeException();
- }
- }
-
- @Override
- public AbstractTaskEvent getCurrentEvent() {
- AbstractTaskEvent e = this.currentEvent;
- this.currentEvent = null;
- return e;
- }
-
- @Override
- public void notifyRecordIsAvailable(int channelIndex) {
- this.availableChannels.add(Integer.valueOf(channelIndex));
-
- RecordAvailabilityListener<T> listener = this.recordAvailabilityListener.get();
- if (listener != null) {
- listener.reportRecordAvailability(this);
- }
- }
-
- /**
- * This method returns the index of a channel which has at least
- * one record available. The method may block until at least one
- * channel has become ready.
- *
- * @return the index of the channel which has at least one record available
- */
- public int waitForAnyChannelToBecomeAvailable() throws InterruptedException {
- return this.availableChannels.take().intValue();
- }
-
-
- @Override
- public boolean isClosed() throws IOException, InterruptedException {
-
- if (this.isClosed) {
- return true;
- }
-
- for (int i = 0; i < this.getNumberOfInputChannels(); i++) {
- final AbstractInputChannel<T> inputChannel = this.inputChannels.get(i);
- if (!inputChannel.isClosed()) {
- return false;
- }
- }
-
- this.isClosed = true;
-
- return true;
- }
-
-
- @Override
- public void close() throws IOException, InterruptedException {
-
- for (int i = 0; i < this.getNumberOfInputChannels(); i++) {
- final AbstractInputChannel<T> inputChannel = this.inputChannels.get(i);
- inputChannel.close();
- }
-
- }
-
-
- @Override
- public String toString() {
- return "Input " + super.toString();
- }
-
-
- @Override
- public void publishEvent(AbstractEvent event) throws IOException, InterruptedException {
-
- // Copy event to all connected channels
- final Iterator<AbstractInputChannel<T>> it = this.inputChannels.iterator();
- while (it.hasNext()) {
- it.next().transferEvent(event);
- }
- }
-
- /**
- * Returns the {@link RecordDeserializerFactory} used by this input gate.
- *
- * @return The {@link RecordDeserializerFactory} used by this input gate.
- */
- public RecordDeserializerFactory<T> getRecordDeserializerFactory() {
- return this.deserializerFactory;
- }
-
-
- @Override
- public void releaseAllChannelResources() {
-
- final Iterator<AbstractInputChannel<T>> it = this.inputChannels.iterator();
- while (it.hasNext()) {
- it.next().releaseAllResources();
- }
- }
-
- @Override
- public void registerRecordAvailabilityListener(final RecordAvailabilityListener<T> listener) {
- if (!this.recordAvailabilityListener.compareAndSet(null, listener)) {
- throw new IllegalStateException(this.recordAvailabilityListener
- + " is already registered as a record availability listener");
- }
- }
-
- public void notifyDataUnitConsumed(int channelIndex) {
- this.channelToReadFrom = -1;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java
deleted file mode 100644
index 20efd94..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RuntimeOutputGate.java
+++ /dev/null
@@ -1,333 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.io.channels.AbstractOutputChannel;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryOutputChannel;
-import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkOutputChannel;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * In Nephele output gates are a specialization of general gates and connect
- * record writers and output channels. As channels, output gates are always
- * parameterized to a specific type of record which they can transport.
- * <p>
- * This class is in general not thread-safe.
- *
- * @param <T>
- * the type of record that can be transported through this gate
- */
-public class RuntimeOutputGate<T extends IOReadableWritable> extends AbstractGate<T> implements OutputGate<T> {
-
- /**
- * The log object used for debugging.
- */
- private static final Log LOG = LogFactory.getLog(OutputGate.class);
-
- /**
- * The list of output channels attached to this gate.
- */
- private final ArrayList<AbstractOutputChannel<T>> outputChannels = new ArrayList<AbstractOutputChannel<T>>();
-
- /**
- * Channel selector to determine which channel is supposed receive the next record.
- */
- private final ChannelSelector<T> channelSelector;
-
- /**
- * The class of the record transported through this output gate.
- */
- private final Class<T> type;
-
- /**
- * Stores whether all records passed to this output gate shall be transmitted through all connected output channels.
- */
- private final boolean isBroadcast;
-
- /**
- * Constructs a new runtime output gate.
- *
- * @param jobID
- * the ID of the job this input gate belongs to
- * @param gateID
- * the ID of the gate
- * @param inputClass
- * the class of the record that can be transported through this
- * gate
- * @param index
- * the index assigned to this output gate at the {@link Environment} object
- * @param channelSelector
- * the channel selector to be used for this output gate
- * @param isBroadcast
- * <code>true</code> if every records passed to this output gate shall be transmitted through all connected
- * output channels, <code>false</code> otherwise
- */
- public RuntimeOutputGate(final JobID jobID, final GateID gateID, final Class<T> inputClass, final int index,
- final ChannelSelector<T> channelSelector, final boolean isBroadcast) {
-
- super(jobID, gateID, index);
-
- this.isBroadcast = isBroadcast;
- this.type = inputClass;
-
- if (this.isBroadcast) {
- this.channelSelector = null;
- } else {
- if (channelSelector == null) {
- this.channelSelector = new DefaultChannelSelector<T>();
- } else {
- this.channelSelector = channelSelector;
- }
- }
- }
-
-
- @Override
- public final Class<T> getType() {
- return this.type;
- }
-
- /**
- * Adds a new output channel to the output gate.
- *
- * @param outputChannel
- * the output channel to be added.
- */
- private void addOutputChannel(AbstractOutputChannel<T> outputChannel) {
- if (!this.outputChannels.contains(outputChannel)) {
- this.outputChannels.add(outputChannel);
- }
- }
-
- /**
- * Removes the output channel with the given ID from the output gate if it
- * exists.
- *
- * @param outputChannelID
- * the ID of the channel to be removed
- */
- public void removeOutputChannel(ChannelID outputChannelID) {
-
- for (int i = 0; i < this.outputChannels.size(); i++) {
-
- final AbstractOutputChannel<T> outputChannel = this.outputChannels.get(i);
- if (outputChannel.getID().equals(outputChannelID)) {
- this.outputChannels.remove(i);
- return;
- }
- }
-
- LOG.debug("Cannot find output channel with ID " + outputChannelID + " to remove");
- }
-
- /**
- * Removes all output channels from the output gate.
- */
- public void removeAllOutputChannels() {
-
- this.outputChannels.clear();
- }
-
-
- @Override
- public boolean isInputGate() {
-
- return false;
- }
-
-
- @Override
- public int getNumberOfOutputChannels() {
-
- return this.outputChannels.size();
- }
-
- /**
- * Returns the output channel from position <code>pos</code> of the gate's
- * internal channel list.
- *
- * @param pos
- * the position to retrieve the channel from
- * @return the channel from the given position or <code>null</code> if such
- * position does not exist.
- */
- public AbstractOutputChannel<T> getOutputChannel(int pos) {
-
- if (pos < this.outputChannels.size()) {
- return this.outputChannels.get(pos);
- } else {
- return null;
- }
- }
-
-
- @Override
- public NetworkOutputChannel<T> createNetworkOutputChannel(final OutputGate<T> outputGate,
- final ChannelID channelID, final ChannelID connectedChannelID) {
-
- final NetworkOutputChannel<T> enoc = new NetworkOutputChannel<T>(outputGate, this.outputChannels.size(),
- channelID, connectedChannelID);
- addOutputChannel(enoc);
-
- return enoc;
- }
-
-
- @Override
- public InMemoryOutputChannel<T> createInMemoryOutputChannel(final OutputGate<T> outputGate,
- final ChannelID channelID, final ChannelID connectedChannelID) {
-
- final InMemoryOutputChannel<T> einoc = new InMemoryOutputChannel<T>(outputGate, this.outputChannels.size(),
- channelID, connectedChannelID);
- addOutputChannel(einoc);
-
- return einoc;
- }
-
-
- @Override
- public void requestClose() throws IOException, InterruptedException {
- // Close all output channels
- for (int i = 0; i < this.getNumberOfOutputChannels(); i++) {
- final AbstractOutputChannel<T> outputChannel = this.getOutputChannel(i);
- outputChannel.requestClose();
- }
- }
-
-
- @Override
- public boolean isClosed() throws IOException, InterruptedException {
-
- boolean allClosed = true;
-
- for (int i = 0; i < this.getNumberOfOutputChannels(); i++) {
- final AbstractOutputChannel<T> outputChannel = this.getOutputChannel(i);
- if (!outputChannel.isClosed()) {
- allClosed = false;
- }
- }
-
- return allClosed;
- }
-
-
- @Override
- public void writeRecord(final T record) throws IOException, InterruptedException {
-
- if (this.isBroadcast) {
-
- if (getChannelType() == ChannelType.INMEMORY) {
-
- final int numberOfOutputChannels = this.outputChannels.size();
- for (int i = 0; i < numberOfOutputChannels; ++i) {
- this.outputChannels.get(i).writeRecord(record);
- }
-
- } else {
-
- // Use optimization for byte buffered channels
- this.outputChannels.get(0).writeRecord(record);
- }
-
- } else {
-
- // Non-broadcast gate, use channel selector to select output channels
- final int numberOfOutputChannels = this.outputChannels.size();
- final int[] selectedOutputChannels = this.channelSelector.selectChannels(record, numberOfOutputChannels);
-
- if (selectedOutputChannels == null) {
- return;
- }
-
-
- for (int i = 0; i < selectedOutputChannels.length; ++i) {
- if (selectedOutputChannels[i] < numberOfOutputChannels) {
- final AbstractOutputChannel<T> outputChannel = this.outputChannels.get(selectedOutputChannels[i]);
- outputChannel.writeRecord(record);
- }
- }
- }
- }
-
-
- @Override
- public List<AbstractOutputChannel<T>> getOutputChannels() {
- return this.outputChannels;
- }
-
-
- @Override
- public String toString() {
- return "Output " + super.toString();
- }
-
-
- @Override
- public void publishEvent(AbstractEvent event) throws IOException, InterruptedException {
-
- // Copy event to all connected channels
- final Iterator<AbstractOutputChannel<T>> it = this.outputChannels.iterator();
- while (it.hasNext()) {
- it.next().transferEvent(event);
- }
- }
-
-
- @Override
- public void flush() throws IOException, InterruptedException {
- // Flush all connected channels
- final Iterator<AbstractOutputChannel<T>> it = this.outputChannels.iterator();
- while (it.hasNext()) {
- it.next().flush();
- }
- }
-
-
- @Override
- public boolean isBroadcast() {
-
- return this.isBroadcast;
- }
-
-
- @Override
- public ChannelSelector<T> getChannelSelector() {
-
- return this.channelSelector;
- }
-
-
- @Override
- public void releaseAllChannelResources() {
-
- final Iterator<AbstractOutputChannel<T>> it = this.outputChannels.iterator();
-
- while (it.hasNext()) {
- it.next().releaseAllResources();
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/UnionRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/UnionRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/UnionRecordReader.java
deleted file mode 100644
index ec8ef0c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/UnionRecordReader.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-public final class UnionRecordReader<T extends IOReadableWritable> extends AbstractUnionRecordReader<T> implements Reader<T> {
-
- private final Class<T> recordType;
-
- private T lookahead;
-
-
- public UnionRecordReader(MutableRecordReader<T>[] recordReaders, Class<T> recordType) {
- super(recordReaders);
- this.recordType = recordType;
- }
-
- @Override
- public boolean hasNext() throws IOException, InterruptedException {
- if (this.lookahead != null) {
- return true;
- } else {
- T record = instantiateRecordType();
- if (getNextRecord(record)) {
- this.lookahead = record;
- return true;
- } else {
- return false;
- }
- }
- }
-
- @Override
- public T next() throws IOException, InterruptedException {
- if (hasNext()) {
- T tmp = this.lookahead;
- this.lookahead = null;
- return tmp;
- } else {
- return null;
- }
- }
-
- private T instantiateRecordType() {
- try {
- return this.recordType.newInstance();
- } catch (InstantiationException e) {
- throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
- } catch (IllegalAccessException e) {
- throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Writer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Writer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Writer.java
deleted file mode 100644
index 91fa3b7..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Writer.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- * A writer that sends records.
- *
- * @param <T> The type of the record that can be emitted with this record writer.
- */
-public interface Writer<T extends IOReadableWritable> {
-
- void emit(T record) throws IOException, InterruptedException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractChannel.java
deleted file mode 100644
index b48e5f0..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractChannel.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * An abstract base class for channel objects.
- */
-public abstract class AbstractChannel {
-
- /**
- * The ID of the channel.
- */
- private final ChannelID channelID;
-
- /**
- * The ID of the connected channel.
- */
- private final ChannelID connectedChannelID;
-
- private final int channelIndex;
-
- /**
- * Auxiliary constructor for channels
- *
- * @param channelIndex
- * the index of the channel in either the output or input gate
- * @param channelID
- * the ID of the channel
- * @param connectedChannelID
- * the ID of the channel this channel is connected to
- * @param compressionLevel
- * the level of compression to be used for this channel
- */
- protected AbstractChannel(final int channelIndex, final ChannelID channelID, final ChannelID connectedChannelID) {
- this.channelIndex = channelIndex;
- this.channelID = channelID;
- this.connectedChannelID = connectedChannelID;
- }
-
- /**
- * Returns the ID of the channel.
- *
- * @return the ID of the channel.
- */
- public ChannelID getID() {
- return this.channelID;
- }
-
- /**
- * Returns the channel's input at the associated gate.
- *
- * @return the channel's input at the associated gate
- */
- public int getChannelIndex() {
- return this.channelIndex;
- }
-
- /**
- * Returns the type of the channel.
- *
- * @return the type of the channel.
- */
- public abstract ChannelType getType();
-
- /**
- * Checks if the channel is closed, i.e. no more records can be transported through the channel.
- *
- * @return <code>true</code> if the channel is closed, <code>false</code> otherwise
- * @throws IOException
- * thrown if an error occurred while closing the channel
- * @throws InterruptedException
- * thrown if the channel is interrupted while waiting for this operation to complete
- */
- public abstract boolean isClosed() throws IOException, InterruptedException;
-
-
- public ChannelID getConnectedChannelID() {
- return this.connectedChannelID;
- }
-
-
- /**
- * Returns the ID of the job this channel belongs to.
- *
- * @return the ID of the job this channel belongs to
- */
- public abstract JobID getJobID();
-
- /**
- * Returns <code>true</code> if this channel is an input channel, <code>false</code> otherwise.
- *
- * @return <code>true</code> if this channel is an input channel, <code>false</code> otherwise
- */
- public abstract boolean isInputChannel();
-
-
- public abstract void transferEvent(AbstractEvent event) throws IOException, InterruptedException;
-
- /**
- * Releases all resources (especially buffers) which are currently allocated by this channel. This method should be
- * called in case of a task error or as a result of a cancel operation.
- */
- public abstract void releaseAllResources();
-
- /**
- * Returns the number of bytes which have been transmitted through this channel since its instantiation.
- *
- * @return the number of bytes which have been transmitted through this channel since its instantiation
- */
- public abstract long getAmountOfDataTransmitted();
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractInputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractInputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractInputChannel.java
deleted file mode 100644
index 4b88aff..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractInputChannel.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.io.InputChannelResult;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * InputChannel is an abstract base class to all different kinds of concrete
- * input channels that can be used. Input channels are always parameterized to
- * a specific type that can be transported through the channel.
-
- * @param <T> The Type of the record that can be transported through the channel.
- */
-public abstract class AbstractInputChannel<T extends IOReadableWritable> extends AbstractChannel {
-
- private final InputGate<T> inputGate;
-
- /**
- * Constructs an input channel with a given input gate associated.
- *
- * @param inputGate
- * the input gate this channel is connected to
- * @param channelIndex
- * the index of the channel in the input gate
- * @param channelID
- * the ID of the channel
- * @param connectedChannelID
- * the ID of the channel this channel is connected to
- * @param compressionLevel
- * the level of compression to be used for this channel
- */
- protected AbstractInputChannel(final InputGate<T> inputGate, final int channelIndex, final ChannelID channelID,
- final ChannelID connectedChannelID) {
- super(channelIndex, channelID, connectedChannelID);
- this.inputGate = inputGate;
- }
-
- /**
- * Returns the input gate associated with the input channel.
- *
- * @return the input gate associated with the input channel.
- */
- public InputGate<T> getInputGate() {
- return this.inputGate;
- }
-
- /**
- * Reads a record from the input channel. If currently no record is available the method
- * returns <code>null</code>. If the channel is closed (i.e. no more records will be received), the method
- * throws an {@link EOFException}.
- *
- * @return a record that has been transported through the channel or <code>null</code> if currently no record is
- * available
- * @throws IOException
- * thrown if the input channel is already closed {@link EOFException} or a transmission error has occurred
- */
- public abstract InputChannelResult readRecord(T target) throws IOException;
-
- /**
- * Immediately closes the input channel. The corresponding output channels are
- * notified if necessary. Any remaining records in any buffers or queue is considered
- * irrelevant and is discarded.
- *
- * @throws InterruptedException
- * thrown if the thread is interrupted while waiting for the channel to close
- * @throws IOException
- * thrown if an I/O error occurs while closing the channel
- */
- public abstract void close() throws IOException, InterruptedException;
-
-
-
- @Override
- public boolean isInputChannel() {
- return true;
- }
-
-
- @Override
- public JobID getJobID() {
- return this.inputGate.getJobID();
- }
-
- public abstract AbstractTaskEvent getCurrentEvent();
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractOutputChannel.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractOutputChannel.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractOutputChannel.java
deleted file mode 100644
index 7974c24..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/AbstractOutputChannel.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-/**
- * OutputChannel is an abstract base class to all different kinds of concrete
- * output channels that can be used. Input channels are always parameterized to
- * a specific type that can be transported through the channel.
- *
- * @param <T>
- * The Type of the record that can be transported through the channel.
- */
-public abstract class AbstractOutputChannel<T extends IOReadableWritable> extends AbstractChannel {
-
- private final OutputGate<T> outputGate;
-
- /**
- * Creates a new output channel object.
- *
- * @param outputGate
- * the output gate this channel is connected to
- * @param channelIndex
- * the index of the channel in the output gate
- * @param channelID
- * the ID of the channel
- * @param connectedChannelID
- * the ID of the channel this channel is connected to
- * @param compressionLevel
- * the level of compression to be used for this channel
- */
- public AbstractOutputChannel(final OutputGate<T> outputGate, final int channelIndex, final ChannelID channelID,
- final ChannelID connectedChannelID) {
- super(channelIndex, channelID, connectedChannelID);
- this.outputGate = outputGate;
- }
-
- /**
- * Returns the output gate this channel is connected to.
- *
- * @return the output gate this channel is connected to
- */
- public OutputGate<T> getOutputGate() {
- return this.outputGate;
- }
-
- /**
- * Writes a record to the channel. The operation may block until the record
- * is completely written to the channel.
- *
- * @param record
- * the record to be written to the channel
- * @throws IOException
- * thrown if an error occurred while transmitting the record
- */
- public abstract void writeRecord(T record) throws IOException, InterruptedException;
-
- /**
- * Requests the output channel to close. After calling this method no more records can be written
- * to the channel. The channel is finally closed when all remaining data that may exist in internal buffers
- * are written to the channel.
- *
- * @throws InterruptedException
- * thrown if the thread is interrupted while requesting the close operation
- * @throws IOException
- * thrown if an I/O error occurs while requesting the close operation
- */
- public abstract void requestClose() throws IOException, InterruptedException;
-
-
-
- @Override
- public boolean isInputChannel() {
- return false;
- }
-
- public abstract void flush() throws IOException, InterruptedException;
-
-
- @Override
- public JobID getJobID() {
- return this.outputGate.getJobID();
- }
-
- /**
- * Returns <code>true</code> if this channel is connected to an output gate which operates in broadcast mode,
- * <code>false</code> otherwise.
- *
- * @return <code>true</code> if the connected output gate operates in broadcase mode, <code>false</code> otherwise
- */
- public boolean isBroadcastChannel() {
-
- return this.outputGate.isBroadcast();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/Buffer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/Buffer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/Buffer.java
deleted file mode 100644
index 80dfa8f..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/Buffer.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import java.io.IOException;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.WritableByteChannel;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-
-/**
- * This class represents the general buffer abstraction that is used by Nephele
- * to transport data through the network or the file system.
- * <p>
- * Buffers may be backed by actual main memory or files.
- * <p>
- * Each buffer is expected to be written and read exactly once. Initially, the every buffer is in write mode. Before
- * reading from the buffer, it must be explicitly switched to read mode.
- * <p>
- * This class is in general not thread-safe.
- *
- */
-public abstract class Buffer implements ReadableByteChannel, WritableByteChannel
-{
- /**
- * Stores whether this buffer has already been recycled.
- */
- private final AtomicBoolean isRecycled = new AtomicBoolean(false);
-
- /**
- * Constructs a new buffer object.
- *
- * @param internalBuffer
- * the concrete implementation which backs the buffer
- */
- protected Buffer()
- {}
-
- /**
- * Reads data from the buffer and writes it to the
- * given {@link WritableByteChannel} object.
- *
- * @param destination The {@link WritableByteChannel} object to write the data to
- * @return The number of bytes read from the buffer, potentially <code>0</code> or <code>-1</code to indicate the
- * end of the stream.
- * @throws IOException Thrown if an error occurs while writing to the {@link WritableByteChannel} object.
- */
-
- public abstract boolean isOpen();
-
-
-
- @Override
- public abstract void close() throws IOException;
-
- /**
- * Reads data from the given {@link ReadableByteChannel} object and
- * writes it to the buffer.
- *
- * @param source The {@link ReadableByteChannel} object to read data from.
- * @return The number of bytes written to the buffer, possibly <code>0</code>.
- * @throws IOException Thrown if an error occurs while writing data to the buffer.
- */
- public abstract int write(ReadableByteChannel source) throws IOException;
-
-
- /**
- * Returns the number of bytes which can be either still written to or read from
- * the buffer, depending whether the buffer is still in write mode or not.
- * <p>
- * If in write mode, the method returns the number of bytes which can be written to be buffer, before its capacity
- * limit is reached. In read mode, the method returns the number of bytes which can be read from the number until
- * all data previously written to the buffer is consumed.
- *
- * @return the number of bytes which can be either written to or read from the buffer
- */
- public abstract int remaining();
-
- /**
- * Checks whether data can still be written to or read from the buffer.
- *
- * @return <code>true</code> if data can be still written to or read from
- * the buffer, <code>false</code> otherwise
- */
- public boolean hasRemaining() {
- return remaining() > 0;
- }
-
- /**
- * Returns the size of the buffer. In write mode, the size of the buffer is the initial capacity
- * of the buffer. In read mode, the size of the buffer is number of bytes which have been
- * previously written to the buffer.
- *
- * @return the size of the buffer in bytes
- */
- public abstract int size();
-
- /**
- * Recycles the buffer. In case of a memory backed buffer, the internal memory buffer
- * is returned to a global buffer queue. In case of a file backed buffer, the temporary
- * file created for this buffer is deleted. A buffer can only be recycled once. Calling this method more than once
- * will therefore have no effect.
- */
- public final void recycleBuffer()
- {
- if (this.isRecycled.compareAndSet(false, true)) {
- recycle();
- }
- }
-
- protected abstract void recycle();
-
-
- /**
- * Returns whether the buffer is backed by main memory or a file.
- *
- * @return <code>true</code> if the buffer is backed by main memory
- * or <code>false</code> if it is backed by a file
- */
- public abstract boolean isBackedByMemory();
-
- /**
- * Copies the content of the buffer to the given destination buffer. The state of the source buffer is not modified
- * by this operation.
- *
- * @param destinationBuffer
- * the destination buffer to copy this buffer's content to
- * @throws IOException
- * thrown if an error occurs while copying the data
- */
- public abstract void copyToBuffer(Buffer destinationBuffer) throws IOException;
-
- /**
- * Duplicates the buffer. This operation does not duplicate the actual
- * content of the buffer, only the reading/writing state. As a result,
- * modifications to the original buffer will affect the duplicate and vice-versa.
- *
- * @return the duplicated buffer
- */
- public abstract Buffer duplicate() throws IOException, InterruptedException;
-
- /**
- * Reads data from the buffer and writes it to the
- * given {@link WritableByteChannel} object.
- *
- * @param destination The {@link WritableByteChannel} object to write the data to
- * @return The number of bytes read from the buffer, potentially <code>0</code> or <code>-1</code to indicate the
- * end of the stream.
- * @throws IOException Thrown if an error occurs while writing to the {@link WritableByteChannel} object.
- */
- public abstract int writeTo(WritableByteChannel writableByteChannel) throws IOException;
-
- /**
- * Flip buffer (exchange limit and position).
- */
- public abstract void flip();
-
- /**
- * Returns the current read/write position for relative operations.
- * @return
- */
- public abstract int position();
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java
deleted file mode 100644
index 001e5a7..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/BufferFactory.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import eu.stratosphere.core.memory.MemorySegment;
-
-
-public final class BufferFactory {
-
- public static MemoryBuffer createFromMemory(final int bufferSize, final MemorySegment byteBuffer,
- final MemoryBufferPoolConnector bufferPoolConnector) {
-
- return new MemoryBuffer(bufferSize, byteBuffer, bufferPoolConnector);
- }
-
- /**
- * Private constructor to prevent instantiation.
- */
- private BufferFactory() {
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java
deleted file mode 100644
index a4e3ba2..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelID.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import eu.stratosphere.nephele.io.AbstractID;
-
-/**
- * A class for statistically unique channel IDs.
- *
- */
-public class ChannelID extends AbstractID {
-
- /**
- * Constructs a new, random channel ID.
- */
- public ChannelID() {
- super();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelType.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelType.java
deleted file mode 100644
index 17fc980..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelType.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-/**
- * An enumeration for declaring the type of channel.
- *
- */
-public enum ChannelType {
-
- /**
- * Enumeration type for network channels.
- */
- NETWORK,
-
- /**
- * Enumeration type for in-memory channels.
- */
- INMEMORY
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java
deleted file mode 100644
index a44b1bd..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithAccessInfo.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import java.nio.channels.FileChannel;
-
-interface ChannelWithAccessInfo {
-
- FileChannel getChannel();
-
- FileChannel getAndIncrementReferences();
-
- /**
- * Increments the references to this channel. Returns <code>true</code>, if successful, and <code>false</code>,
- * if the channel has been disposed in the meantime.
- *
- * @return True, if successful, false, if the channel has been disposed.
- */
- boolean incrementReferences();
-
- ChannelWithPosition reserveWriteSpaceAndIncrementReferences(int spaceToReserve);
-
- /**
- * Decrements the number of references to this channel. If the number of references is zero after the
- * decrement, the channel is deleted.
- *
- * @return The number of references remaining after the decrement.
- * @throws IllegalStateException
- * Thrown, if the number of references is already zero or below.
- */
- int decrementReferences();
-
- /**
- * Disposes the channel without further notice. Tries to close it (swallowing all exceptions) and tries
- * to delete the file.
- */
- void disposeSilently();
-
- /**
- * Updates the flag which indicates whether the underlying physical file shall be deleted when it is closed. Once
- * the flag was updated to <code>false</code> it cannot be set to <code>true</code> again.
- *
- * @param deleteOnClose
- * <code>true</code> to indicate the file shall be deleted when closed, <code>false</code> otherwise
- */
- void updateDeleteOnCloseFlag(final boolean deleteOnClose);
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithPosition.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithPosition.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithPosition.java
deleted file mode 100644
index 5678439..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/ChannelWithPosition.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import java.nio.channels.FileChannel;
-
-/**
- * A simple encapsulation of a file channel with an offset. This object is used for purposes, where
- * the channel is accessed by multiple threads and its internal position may be changed.
- */
-public class ChannelWithPosition {
-
- private final FileChannel channel;
-
- private final long offset;
-
- ChannelWithPosition(final FileChannel channel, final long offset) {
- this.channel = channel;
- this.offset = offset;
- }
-
- public FileChannel getChannel() {
-
- return this.channel;
- }
-
- public long getOffset() {
-
- return this.offset;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DefaultDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DefaultDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DefaultDeserializer.java
deleted file mode 100644
index 3f47fe1..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DefaultDeserializer.java
+++ /dev/null
@@ -1,781 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.UTFDataFormatException;
-import java.nio.BufferUnderflowException;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.channels.ReadableByteChannel;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.core.memory.DataInputView;
-import eu.stratosphere.nephele.io.RecordDeserializer;
-
-/**
- * A class for deserializing a portion of binary data into records of type <code>T</code>. The internal
- * buffer grows dynamically to the size that is required for deserialization.
- *
- * @param <T>
- * The type of the record this deserialization buffer can be used for.
- */
-public class DefaultDeserializer<T extends IOReadableWritable> implements RecordDeserializer<T> {
- /**
- * The size of an integer in byte.
- */
- private static final int SIZEOFINT = 4;
-
- // --------------------------------------------------------------------------------------------
-
- /**
- * The data input buffer used for deserialization.
- */
- private final DataInputWrapper deserializationWrapper;
-
- /**
- * Buffer to reconstruct the length field.
- */
- private final ByteBuffer lengthBuf;
-
- /**
- * Temporary buffer.
- */
- private ByteBuffer tempBuffer;
-
- /**
- * The type of the record to be deserialized.
- */
- private final Class<? extends T> recordType;
-
- /**
- * Size of the record to be deserialized in bytes.
- */
- private int recordLength = -1;
-
- /**
- * Flag indicating whether to throw an exception if nothing can be read any more.
- */
- private final boolean propagateEndOfStream;
-
- // --------------------------------------------------------------------------------------------
-
- /**
- * Constructs a new deserialization buffer with the specified type.
- *
- * @param recordType
- * The type of the record to be deserialized.
- */
- public DefaultDeserializer(final Class<? extends T> recordType) {
- this(recordType, false);
- }
-
- /**
- * Constructs a new deserialization buffer with the specified type.
- *
- * @param recordType
- * The type of the record to be deserialized.
- * @param propagateEndOfStream
- * <code>True</code>, if end of stream notifications during the
- * deserialization process shall be propagated to the caller, <code>false</code> otherwise.
- */
- public DefaultDeserializer(final Class<? extends T> recordType, final boolean propagateEndOfStream) {
- this.recordType = recordType;
- this.propagateEndOfStream = propagateEndOfStream;
-
- this.lengthBuf = ByteBuffer.allocate(SIZEOFINT);
- this.lengthBuf.order(ByteOrder.BIG_ENDIAN);
-
- this.tempBuffer = ByteBuffer.allocate(128);
- this.tempBuffer.order(ByteOrder.BIG_ENDIAN);
-
- this.deserializationWrapper = new DataInputWrapper();
- this.deserializationWrapper.setArray(this.tempBuffer.array());
- }
-
- // --------------------------------------------------------------------------------------------
-
- /*
- * (non-Javadoc)
- * @see eu.stratosphere.nephele.io.RecordDeserializer#readData(java.lang.Object,
- * java.nio.channels.ReadableByteChannel)
- */
- @Override
- public T readData(T target, final ReadableByteChannel readableByteChannel) throws IOException {
- // check whether the length has already been de-serialized
- final int len;
- if (this.recordLength < 0) {
- if (readableByteChannel.read(this.lengthBuf) == -1 && this.propagateEndOfStream) {
- if (this.lengthBuf.position() == 0) {
- throw new EOFException();
- } else {
- throw new IOException("Deserialization error: Expected to read " + this.lengthBuf.remaining()
- + " more bytes of length information from the stream!");
- }
- }
-
- if (this.lengthBuf.hasRemaining()) {
- return null;
- }
-
- len = this.lengthBuf.getInt(0);
- this.lengthBuf.clear();
-
- if (this.tempBuffer.capacity() < len) {
- this.tempBuffer = ByteBuffer.allocate(len);
- this.tempBuffer.order(ByteOrder.BIG_ENDIAN);
- this.deserializationWrapper.setArray(this.tempBuffer.array());
- }
-
- // Important: limit the number of bytes that can be read into the buffer
- this.tempBuffer.position(0);
- this.tempBuffer.limit(len);
- } else {
- len = this.recordLength;
- }
-
- if (readableByteChannel.read(this.tempBuffer) == -1 && this.propagateEndOfStream) {
- throw new IOException("Deserilization error: Expected to read " + this.tempBuffer.remaining()
- + " more bytes from stream!");
- }
-
- if (this.tempBuffer.hasRemaining()) {
- this.recordLength = len;
- return null;
- } else {
- this.recordLength = -1;
- }
-
- this.deserializationWrapper.reset(len);
-
- if (target == null) {
- target = instantiateTarget();
- }
-
- // now de-serialize the target
- try {
- target.read(this.deserializationWrapper);
- return target;
- } catch (BufferUnderflowException buex) {
- throw new EOFException();
- }
- }
-
- private final T instantiateTarget() throws IOException {
- try {
- return this.recordType.newInstance();
- } catch (Exception e) {
- throw new IOException("Could not instantiate the given record type: " + e.getMessage(), e);
- }
- }
-
- /*
- * (non-Javadoc)
- * @see eu.stratosphere.nephele.io.RecordDeserializer#clear()
- */
- @Override
- public void clear() {
-
- this.recordLength = -1;
- if (this.tempBuffer != null) {
- this.tempBuffer.clear();
- }
- if (this.lengthBuf != null) {
- this.lengthBuf.clear();
- }
- }
-
- /*
- * (non-Javadoc)
- * @see eu.stratosphere.nephele.io.RecordDeserializer#hasUnfinishedData()
- */
- @Override
- public boolean hasUnfinishedData() {
- if (this.recordLength != -1) {
- return true;
- }
-
- if (this.lengthBuf.position() > 0) {
- return true;
- }
-
- return false;
- }
-
- // --------------------------------------------------------------------------------------------
-
- // private static final class DataInputWrapper implements DataInputView
- // {
- // private ByteBuffer source;
- //
- // private byte[] utfByteBuffer; // reusable byte buffer for utf-8 decoding
- // private char[] utfCharBuffer; // reusable char buffer for utf-8 decoding
- //
- //
- // void set(ByteBuffer source) {
- // this.source = source;
- // }
- //
- //
- // /* (non-Javadoc)
- // * @see java.io.DataInput#readFully(byte[])
- // */
- // @Override
- // public void readFully(byte[] b) {
- // this.source.get(b);
- // }
- //
- // /* (non-Javadoc)
- // * @see java.io.DataInput#readFully(byte[], int, int)
- // */
- // @Override
- // public void readFully(byte[] b, int off, int len) {
- // this.source.get(b, off, len);
- // }
- //
- // /* (non-Javadoc)
- // * @see java.io.DataInput#skipBytes(int)
- // */
- // @Override
- // public int skipBytes(int n) {
- // int newPos = this.source.position() + n;
- // if (newPos > this.source.limit()) {
- // newPos = this.source.limit();
- // n = newPos - this.source.position();
- // }
- // this.source.position(newPos);
- // return n;
- // }
- //
- // /* (non-Javadoc)
- // * @see java.io.DataInput#readBoolean()
- // */
- // @Override
- // public boolean readBoolean() {
- // return this.source.get() != 0;
- // }
- //
- // /* (non-Javadoc)
- // * @see java.io.DataInput#readByte()
- // */
- // @Override
- // public byte readByte() {
- // return this.source.get();
- // }
- //
- // /* (non-Javadoc)
- // * @see java.io.DataInput#readUnsignedByte()
- // */
- // @Override
- // public int readUnsignedByte() {
- // return this.source.get() & 0xff;
- // }
- //
- // /* (non-Javadoc)
- // * @see java.io.DataInput#readShort()
- // */
- // @Override
- // public short readShort() {
- // return this.source.getShort();
- // }
- //
- // /* (non-Javadoc)
- // * @see java.io.DataInput#readUnsignedShort()
- // */
- // @Override
- // public int readUnsignedShort() {
- // return this.source.getShort() & 0xffff;
- // }
- //
- // /* (non-Javadoc)
- // * @see java.io.DataInput#readChar()
- // */
- // @Override
- // public char readChar() {
- // return this.source.getChar();
- // }
- //
- // /* (non-Javadoc)
- // * @see java.io.DataInput#readInt()
- // */
- // @Override
- // public int readInt() {
- // return this.source.getInt();
- // }
- //
- // /* (non-Javadoc)
- // * @see java.io.DataInput#readLong()
- // */
- // @Override
- // public long readLong() {
- // return this.source.getLong();
- // }
- //
- // /* (non-Javadoc)
- // * @see java.io.DataInput#readFloat()
- // */
- // @Override
- // public float readFloat() {
- // return Float.intBitsToFloat(this.source.getInt());
- // }
- //
- // /* (non-Javadoc)
- // * @see java.io.DataInput#readDouble()
- // */
- // @Override
- // public double readDouble() {
- // return Double.longBitsToDouble(this.source.getLong());
- // }
- //
- // /* (non-Javadoc)
- // * @see java.io.DataInput#readLine()
- // */
- // @Override
- // public String readLine()
- // {
- // if (this.source.hasRemaining()) {
- // // read until a newline is found
- // StringBuilder bld = new StringBuilder();
- // char curr;
- // while (this.source.hasRemaining() && (curr = (char) readUnsignedByte()) != '\n') {
- // bld.append(curr);
- // }
- // // trim a trailing carriage return
- // int len = bld.length();
- // if (len > 0 && bld.charAt(len - 1) == '\r') {
- // bld.setLength(len - 1);
- // }
- // String s = bld.toString();
- // bld.setLength(0);
- // return s;
- // } else {
- // return null;
- // }
- // }
- //
- // /* (non-Javadoc)
- // * @see java.io.DataInput#readUTF()
- // */
- // @Override
- // public String readUTF() throws IOException
- // {
- // final int utflen = readUnsignedShort();
- //
- // final byte[] bytearr;
- // final char[] chararr;
- //
- // if (this.utfByteBuffer == null || this.utfByteBuffer.length < utflen) {
- // bytearr = new byte[utflen];
- // this.utfByteBuffer = bytearr;
- // } else {
- // bytearr = this.utfByteBuffer;
- // }
- // if (this.utfCharBuffer == null || this.utfCharBuffer.length < utflen) {
- // chararr = new char[utflen];
- // this.utfCharBuffer = chararr;
- // } else {
- // chararr = this.utfCharBuffer;
- // }
- //
- // int c, char2, char3;
- // int count = 0;
- // int chararr_count = 0;
- //
- // readFully(bytearr, 0, utflen);
- //
- // while (count < utflen) {
- // c = (int) bytearr[count] & 0xff;
- // if (c > 127)
- // break;
- // count++;
- // chararr[chararr_count++] = (char) c;
- // }
- //
- // while (count < utflen) {
- // c = (int) bytearr[count] & 0xff;
- // switch (c >> 4) {
- // case 0:
- // case 1:
- // case 2:
- // case 3:
- // case 4:
- // case 5:
- // case 6:
- // case 7:
- // /* 0xxxxxxx */
- // count++;
- // chararr[chararr_count++] = (char) c;
- // break;
- // case 12:
- // case 13:
- // /* 110x xxxx 10xx xxxx */
- // count += 2;
- // if (count > utflen)
- // throw new UTFDataFormatException("malformed input: partial character at end");
- // char2 = (int) bytearr[count - 1];
- // if ((char2 & 0xC0) != 0x80)
- // throw new UTFDataFormatException("malformed input around byte " + count);
- // chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
- // break;
- // case 14:
- // /* 1110 xxxx 10xx xxxx 10xx xxxx */
- // count += 3;
- // if (count > utflen)
- // throw new UTFDataFormatException("malformed input: partial character at end");
- // char2 = (int) bytearr[count - 2];
- // char3 = (int) bytearr[count - 1];
- // if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80))
- // throw new UTFDataFormatException("malformed input around byte " + (count - 1));
- // chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
- // break;
- // default:
- // /* 10xx xxxx, 1111 xxxx */
- // throw new UTFDataFormatException("malformed input around byte " + count);
- // }
- // }
- // // The number of chars produced may be less than utflen
- // return new String(chararr, 0, chararr_count);
- // }
- //
- //
- // /* (non-Javadoc)
- // * @see eu.stratosphere.nephele.services.memorymanager.DataInputView#skipBytesToRead(int)
- // */
- // @Override
- // public void skipBytesToRead(int numBytes) throws IOException {
- // if (this.source.remaining() < numBytes) {
- // throw new EOFException();
- // } else {
- // this.source.position(this.source.position() + numBytes);
- // }
- // }
- // }
-
- private static final class DataInputWrapper implements DataInputView {
- private byte[] source;
-
- private int position;
-
- private int limit;
-
- private char[] utfCharBuffer; // reusable char buffer for utf-8 decoding
-
- void setArray(byte[] source) {
- this.source = source;
- }
-
- void reset(int limit) {
- this.position = 0;
- this.limit = limit;
- }
-
- /*
- * (non-Javadoc)
- * @see java.io.DataInput#readFully(byte[])
- */
- @Override
- public void readFully(byte[] b) throws EOFException {
- readFully(b, 0, b.length);
- }
-
- /*
- * (non-Javadoc)
- * @see java.io.DataInput#readFully(byte[], int, int)
- */
- @Override
- public void readFully(byte[] b, int off, int len) throws EOFException {
- if (this.position <= this.limit - len) {
- System.arraycopy(this.source, this.position, b, off, len);
- this.position += len;
- } else {
- throw new EOFException();
- }
- }
-
- /*
- * (non-Javadoc)
- * @see java.io.DataInput#skipBytes(int)
- */
- @Override
- public int skipBytes(int n) {
- if (n < 0) {
- throw new IllegalArgumentException("Number of bytes to skip must not be negative.");
- }
-
- int toSkip = Math.min(this.limit - this.position, n);
- this.position += toSkip;
- return toSkip;
- }
-
- /*
- * (non-Javadoc)
- * @see java.io.DataInput#readBoolean()
- */
- @Override
- public boolean readBoolean() throws EOFException {
- return readByte() != 0;
- }
-
- /*
- * (non-Javadoc)
- * @see java.io.DataInput#readByte()
- */
- @Override
- public byte readByte() throws EOFException {
- if (this.position < this.limit) {
- return this.source[this.position++];
- } else {
- throw new EOFException();
- }
- }
-
- /*
- * (non-Javadoc)
- * @see java.io.DataInput#readUnsignedByte()
- */
- @Override
- public int readUnsignedByte() throws EOFException {
- return readByte() & 0xff;
- }
-
- /*
- * (non-Javadoc)
- * @see java.io.DataInput#readShort()
- */
- @Override
- public short readShort() throws EOFException {
- if (this.position < this.limit - 1) {
- short num = (short) (
- ((this.source[this.position + 0] & 0xff) << 8) |
- ((this.source[this.position + 1] & 0xff)));
- this.position += 2;
- return num;
- } else {
- throw new EOFException();
- }
- }
-
- /*
- * (non-Javadoc)
- * @see java.io.DataInput#readUnsignedShort()
- */
- @Override
- public int readUnsignedShort() throws EOFException {
- return readShort() & 0xffff;
- }
-
- /*
- * (non-Javadoc)
- * @see java.io.DataInput#readChar()
- */
- @Override
- public char readChar() throws EOFException {
- if (this.position < this.limit - 1) {
- char c = (char) (
- ((this.source[this.position + 0] & 0xff) << 8) |
- ((this.source[this.position + 1] & 0xff)));
- this.position += 2;
- return c;
- } else {
- throw new EOFException();
- }
- }
-
- /*
- * (non-Javadoc)
- * @see java.io.DataInput#readInt()
- */
- @Override
- public int readInt() throws EOFException {
- if (this.position < this.limit - 3) {
- final int num = ((this.source[this.position + 0] & 0xff) << 24) |
- ((this.source[this.position + 1] & 0xff) << 16) |
- ((this.source[this.position + 2] & 0xff) << 8) |
- ((this.source[this.position + 3] & 0xff));
- this.position += 4;
- return num;
- } else {
- throw new EOFException();
- }
- }
-
- /*
- * (non-Javadoc)
- * @see java.io.DataInput#readLong()
- */
- @Override
- public long readLong() throws EOFException {
- if (this.position < this.limit - 7) {
- final long num = (((long) this.source[this.position + 0] & 0xff) << 56) |
- (((long) this.source[this.position + 1] & 0xff) << 48) |
- (((long) this.source[this.position + 2] & 0xff) << 40) |
- (((long) this.source[this.position + 3] & 0xff) << 32) |
- (((long) this.source[this.position + 4] & 0xff) << 24) |
- (((long) this.source[this.position + 5] & 0xff) << 16) |
- (((long) this.source[this.position + 6] & 0xff) << 8) |
- (((long) this.source[this.position + 7] & 0xff) << 0);
- this.position += 8;
- return num;
- } else {
- throw new EOFException();
- }
- }
-
- /*
- * (non-Javadoc)
- * @see java.io.DataInput#readFloat()
- */
- @Override
- public float readFloat() throws EOFException {
- return Float.intBitsToFloat(readInt());
- }
-
- /*
- * (non-Javadoc)
- * @see java.io.DataInput#readDouble()
- */
- @Override
- public double readDouble() throws EOFException {
- return Double.longBitsToDouble(readLong());
- }
-
- /*
- * (non-Javadoc)
- * @see java.io.DataInput#readLine()
- */
- @Override
- public String readLine() {
- if (this.position < this.limit) {
- // read until a newline is found
- StringBuilder bld = new StringBuilder();
- char curr;
- while (this.position < this.limit && (curr = (char) (this.source[this.position++] & 0xff)) != '\n') {
- bld.append(curr);
- }
- // trim a trailing carriage return
- int len = bld.length();
- if (len > 0 && bld.charAt(len - 1) == '\r') {
- bld.setLength(len - 1);
- }
- String s = bld.toString();
- bld.setLength(0);
- return s;
- } else {
- return null;
- }
- }
-
- /*
- * (non-Javadoc)
- * @see java.io.DataInput#readUTF()
- */
- @Override
- public String readUTF() throws IOException {
- final int utflen = readUnsignedShort();
- final int utfLimit = this.position + utflen;
-
- if (utfLimit > this.limit) {
- throw new EOFException();
- }
-
- final byte[] bytearr = this.source;
- final char[] chararr;
- if (this.utfCharBuffer == null || this.utfCharBuffer.length < utflen) {
- chararr = new char[utflen];
- this.utfCharBuffer = chararr;
- } else {
- chararr = this.utfCharBuffer;
- }
-
- int c, char2, char3;
- int count = this.position;
- int chararr_count = 0;
-
- while (count < utfLimit) {
- c = (int) bytearr[count] & 0xff;
- if (c > 127) {
- break;
- }
- count++;
- chararr[chararr_count++] = (char) c;
- }
-
- while (count < utfLimit) {
- c = (int) bytearr[count] & 0xff;
- switch (c >> 4) {
- case 0:
- case 1:
- case 2:
- case 3:
- case 4:
- case 5:
- case 6:
- case 7:
- /* 0xxxxxxx */
- count++;
- chararr[chararr_count++] = (char) c;
- break;
- case 12:
- case 13:
- /* 110x xxxx 10xx xxxx */
- count += 2;
- if (count > utfLimit) {
- throw new UTFDataFormatException("Malformed input: partial character at end");
- }
- char2 = (int) bytearr[count - 1];
- if ((char2 & 0xC0) != 0x80) {
- throw new UTFDataFormatException("Malformed input around byte " + count);
- }
- chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
- break;
- case 14:
- /* 1110 xxxx 10xx xxxx 10xx xxxx */
- count += 3;
- if (count > utfLimit) {
- throw new UTFDataFormatException("Malformed input: partial character at end");
- }
- char2 = (int) bytearr[count - 2];
- char3 = (int) bytearr[count - 1];
- if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
- throw new UTFDataFormatException("Malformed input around byte " + (count - 1));
- }
- chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
- break;
- default:
- /* 10xx xxxx, 1111 xxxx */
- throw new UTFDataFormatException("Malformed input around byte " + count);
- }
- }
- // The number of chars produced may be less than utflen
- this.position += utflen;
- return new String(chararr, 0, chararr_count);
- }
-
- /*
- * (non-Javadoc)
- * @see eu.stratosphere.nephele.services.memorymanager.DataInputView#skipBytesToRead(int)
- */
- @Override
- public void skipBytesToRead(int numBytes) throws EOFException {
- if (numBytes < 0) {
- throw new IllegalArgumentException("Number of bytes to skip must not be negative.");
- } else if (this.limit - this.position < numBytes) {
- throw new EOFException();
- } else {
- this.position += numBytes;
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java
deleted file mode 100644
index 6873c44..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/channels/DistributedChannelWithAccessInfo.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import java.io.IOException;
-import java.nio.channels.FileChannel;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.fs.FileChannelWrapper;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.core.fs.Path;
-
-final class DistributedChannelWithAccessInfo implements ChannelWithAccessInfo {
-
- /**
- * The logging object.
- */
- private static final Log LOG = LogFactory.getLog(DistributedChannelWithAccessInfo.class);
-
- private final FileSystem fs;
-
- private final Path checkpointFile;
-
- private final FileChannelWrapper channel;
-
- private final AtomicLong reservedWritePosition;
-
- private final AtomicInteger referenceCounter;
-
- private final AtomicBoolean deleteOnClose;
-
- DistributedChannelWithAccessInfo(final FileSystem fs, final Path checkpointFile, final int bufferSize,
- final boolean deleteOnClose) throws IOException {
-
- this.fs = fs;
- this.checkpointFile = checkpointFile;
- this.channel = new FileChannelWrapper(fs, checkpointFile, bufferSize, (short) 2);
- this.reservedWritePosition = new AtomicLong(0L);
- this.referenceCounter = new AtomicInteger(0);
- this.deleteOnClose = new AtomicBoolean(deleteOnClose);
- }
-
-
- @Override
- public FileChannel getChannel() {
-
- return this.channel;
- }
-
-
- @Override
- public FileChannel getAndIncrementReferences() {
-
- if (incrementReferences()) {
- return this.channel;
- } else {
- return null;
- }
- }
-
- @Override
- public ChannelWithPosition reserveWriteSpaceAndIncrementReferences(final int spaceToReserve) {
-
- if (incrementReferences()) {
- return new ChannelWithPosition(this.channel, this.reservedWritePosition.getAndAdd(spaceToReserve));
- } else {
- return null;
- }
- }
-
-
- @Override
- public int decrementReferences() {
-
- int current = this.referenceCounter.get();
- while (true) {
- if (current <= 0) {
- // this is actually an error case, because the channel was deleted before
- throw new IllegalStateException("The references to the file were already at zero.");
- }
-
- if (current == 1) {
- // this call decrements to zero, so mark it as deleted
- if (this.referenceCounter.compareAndSet(current, Integer.MIN_VALUE)) {
- current = 0;
- break;
- }
- } else if (this.referenceCounter.compareAndSet(current, current - 1)) {
- current = current - 1;
- break;
- }
- current = this.referenceCounter.get();
- }
-
- if (current > 0) {
- return current;
- } else if (current == 0) {
- // delete the channel
- this.referenceCounter.set(Integer.MIN_VALUE);
- this.reservedWritePosition.set(Long.MIN_VALUE);
- try {
- this.channel.close();
- if (this.deleteOnClose.get()) {
- this.fs.delete(this.checkpointFile, false);
- }
-
- } catch (IOException ioex) {
- if (LOG.isErrorEnabled()) {
- LOG.error("Error while closing spill file for file buffers: " + ioex.getMessage(), ioex);
- }
- }
- return current;
- } else {
- throw new IllegalStateException("The references to the file were already at zero.");
- }
- }
-
-
- @Override
- public boolean incrementReferences() {
-
- int current = this.referenceCounter.get();
- while (true) {
- // check whether it was disposed in the meantime
- if (current < 0) {
- return false;
- }
- // atomically check and increment
- if (this.referenceCounter.compareAndSet(current, current + 1)) {
- return true;
- }
- current = this.referenceCounter.get();
- }
- }
-
-
- @Override
- public void disposeSilently() {
-
- this.referenceCounter.set(Integer.MIN_VALUE);
- this.reservedWritePosition.set(Long.MIN_VALUE);
-
- if (this.channel.isOpen()) {
- try {
- this.channel.close();
- if (this.deleteOnClose.get()) {
- this.fs.delete(this.checkpointFile, false);
- }
- } catch (Throwable t) {
- }
- }
- }
-
-
- @Override
- public void updateDeleteOnCloseFlag(final boolean deleteOnClose) {
-
- this.deleteOnClose.compareAndSet(true, deleteOnClose);
- }
-}
[03/30] git commit: Fix dependency exclusion for examples in
yarn-uberjar
Posted by rm...@apache.org.
Fix dependency exclusion for examples in yarn-uberjar
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/0bb812e4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/0bb812e4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/0bb812e4
Branch: refs/heads/master
Commit: 0bb812e431f266a74ff9a668e055209ba5122c7d
Parents: 2a9ed09
Author: mingliang <qm...@gmail.com>
Authored: Fri Jun 6 15:43:42 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Fri Jun 6 16:00:59 2014 +0200
----------------------------------------------------------------------
stratosphere-dist/src/main/assemblies/yarn-uberjar.xml | 3 ++-
1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/0bb812e4/stratosphere-dist/src/main/assemblies/yarn-uberjar.xml
----------------------------------------------------------------------
diff --git a/stratosphere-dist/src/main/assemblies/yarn-uberjar.xml b/stratosphere-dist/src/main/assemblies/yarn-uberjar.xml
index 5803963..598936f 100644
--- a/stratosphere-dist/src/main/assemblies/yarn-uberjar.xml
+++ b/stratosphere-dist/src/main/assemblies/yarn-uberjar.xml
@@ -31,7 +31,8 @@
<unpack>true</unpack>
<scope>runtime</scope>
<excludes>
- <exclude>eu.stratosphere:stratosphere-examples:*</exclude>
+ <exclude>eu.stratosphere:stratosphere-java-examples:*</exclude>
+ <exclude>eu.stratosphere:stratosphere-scala-examples:*</exclude>
</excludes>
</dependencySet>
</dependencySets>
[25/30] git commit: Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
Offer buffer-oriented API for I/O (#25)
https://github.com/stratosphere/stratosphere/issues/25
The runtime offered a record-oriented API for data transfers, which
* resulted in unnecessary data (de)serialization,
* complicated the upcoming fault tolerance implementation, and
* blocked more efficient implementations of higher-level operators.
With this commit, the runtime offers a buffer-oriented API for the
output side (sending), which is oblivious to records. The buffer
oriented input side (receiving) is still to be implemented.
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/2db78a8d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/2db78a8d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/2db78a8d
Branch: refs/heads/master
Commit: 2db78a8dc1a4664f3e384005d7e07bea594b835b
Parents: 0bb812e
Author: uce <u....@fu-berlin.de>
Authored: Tue Jan 14 17:45:21 2014 +0100
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Sat Jun 7 09:41:21 2014 +0200
----------------------------------------------------------------------
.../spargel/java/record/SpargelIteration.java | 3 +-
.../plantranslate/NepheleJobGraphGenerator.java | 13 +-
.../configuration/ConfigConstants.java | 6 -
.../stratosphere/core/memory/MemorySegment.java | 5 +
.../main/java/eu/stratosphere/types/Record.java | 2 +-
.../eu/stratosphere/nephele/AbstractID.java | 177 ++++
.../nephele/annotations/ForceCheckpoint.java | 30 -
.../nephele/annotations/Stateful.java | 28 -
.../nephele/annotations/Stateless.java | 28 -
.../nephele/annotations/TaskAnnotation.java | 29 -
.../deployment/ChannelDeploymentDescriptor.java | 2 +-
.../deployment/GateDeploymentDescriptor.java | 4 +-
.../nephele/example/events/EventExample.java | 95 ---
.../nephele/example/events/EventReceiver.java | 49 --
.../nephele/example/events/EventSender.java | 68 --
.../nephele/example/events/MyEventListener.java | 46 --
.../stratosphere/nephele/example/grep/Grep.java | 108 ---
.../nephele/example/grep/GrepTask.java | 47 --
.../nephele/example/speedtest/SpeedTest.java | 224 -----
.../example/speedtest/SpeedTestConsumer.java | 45 -
.../example/speedtest/SpeedTestForwarder.java | 54 --
.../example/speedtest/SpeedTestProducer.java | 54 --
.../example/speedtest/SpeedTestRecord.java | 55 --
.../nephele/example/union/ConsumerTask.java | 44 -
.../nephele/example/union/ProducerTask.java | 43 -
.../nephele/example/union/UnionJob.java | 104 ---
.../nephele/example/union/UnionTask.java | 45 -
.../nephele/execution/CancelTaskException.java | 25 +
.../nephele/execution/Environment.java | 82 +-
.../execution/ExecutionStateTransition.java | 24 +-
.../nephele/execution/RuntimeEnvironment.java | 326 +++-----
.../DistributionPatternProvider.java | 2 +-
.../nephele/executiongraph/ExecutionEdge.java | 11 +-
.../nephele/executiongraph/ExecutionGate.java | 4 +-
.../nephele/executiongraph/ExecutionGraph.java | 20 +-
.../executiongraph/ExecutionGroupEdge.java | 28 +-
.../executiongraph/ExecutionGroupVertex.java | 27 +-
.../nephele/executiongraph/ExecutionStage.java | 6 +-
.../nephele/executiongraph/ExecutionVertex.java | 38 +-
.../executiongraph/ExecutionVertexID.java | 2 +-
.../executiongraph/ManagementGraphFactory.java | 2 +-
.../nephele/instance/AbstractInstance.java | 23 +-
.../nephele/instance/AllocationID.java | 2 +-
.../instance/InstanceConnectionInfo.java | 25 +-
.../nephele/instance/InstanceID.java | 2 +-
.../instance/cluster/ClusterManager.java | 10 +-
.../stratosphere/nephele/io/AbstractGate.java | 148 ----
.../eu/stratosphere/nephele/io/AbstractID.java | 214 -----
.../nephele/io/AbstractRecordReader.java | 98 ---
.../nephele/io/AbstractRecordWriter.java | 140 ----
.../io/AbstractSingleGateRecordReader.java | 77 --
.../nephele/io/AbstractUnionRecordReader.java | 152 ----
.../nephele/io/BroadcastRecordWriter.java | 53 --
.../nephele/io/ChannelSelector.java | 39 -
.../nephele/io/DataOutputBuffer.java | 165 ----
.../nephele/io/DefaultChannelSelector.java | 47 --
.../nephele/io/DistributionPattern.java | 33 -
.../java/eu/stratosphere/nephele/io/Gate.java | 126 ---
.../java/eu/stratosphere/nephele/io/GateID.java | 22 -
.../io/ImmutableRecordDeserializerFactory.java | 41 -
.../nephele/io/InputChannelResult.java | 23 -
.../eu/stratosphere/nephele/io/InputGate.java | 136 ----
.../stratosphere/nephele/io/MutableReader.java | 32 -
.../io/MutableRecordDeserializerFactory.java | 52 --
.../nephele/io/MutableRecordReader.java | 119 ---
.../nephele/io/MutableUnionRecordReader.java | 37 -
.../eu/stratosphere/nephele/io/OutputGate.java | 149 ----
.../java/eu/stratosphere/nephele/io/Reader.java | 30 -
.../eu/stratosphere/nephele/io/ReaderBase.java | 67 --
.../nephele/io/RecordAvailabilityListener.java | 35 -
.../nephele/io/RecordDeserializer.java | 55 --
.../nephele/io/RecordDeserializerFactory.java | 31 -
.../stratosphere/nephele/io/RecordReader.java | 153 ----
.../stratosphere/nephele/io/RecordWriter.java | 82 --
.../nephele/io/RuntimeInputGate.java | 330 --------
.../nephele/io/RuntimeOutputGate.java | 333 --------
.../nephele/io/UnionRecordReader.java | 67 --
.../java/eu/stratosphere/nephele/io/Writer.java | 28 -
.../nephele/io/channels/AbstractChannel.java | 127 ---
.../io/channels/AbstractInputChannel.java | 102 ---
.../io/channels/AbstractOutputChannel.java | 111 ---
.../nephele/io/channels/Buffer.java | 175 ----
.../nephele/io/channels/BufferFactory.java | 32 -
.../nephele/io/channels/ChannelID.java | 30 -
.../nephele/io/channels/ChannelType.java | 31 -
.../io/channels/ChannelWithAccessInfo.java | 58 --
.../io/channels/ChannelWithPosition.java | 42 -
.../io/channels/DefaultDeserializer.java | 781 ------------------
.../DistributedChannelWithAccessInfo.java | 176 ----
.../io/channels/LocalChannelWithAccessInfo.java | 170 ----
.../nephele/io/channels/MemoryBuffer.java | 249 ------
.../io/channels/MemoryBufferPoolConnector.java | 32 -
.../io/channels/MemoryBufferRecycler.java | 91 ---
.../io/channels/SerializationBuffer.java | 141 ----
.../AbstractByteBufferedInputChannel.java | 243 ------
.../AbstractByteBufferedOutputChannel.java | 255 ------
.../io/channels/bytebuffered/BufferOrEvent.java | 52 --
.../ByteBufferedChannelCloseEvent.java | 35 -
.../ByteBufferedInputChannelBroker.java | 36 -
.../ByteBufferedOutputChannelBroker.java | 71 --
.../bytebuffered/EndOfSuperstepEvent.java | 34 -
.../bytebuffered/InMemoryInputChannel.java | 34 -
.../bytebuffered/InMemoryOutputChannel.java | 34 -
.../bytebuffered/NetworkInputChannel.java | 34 -
.../bytebuffered/NetworkOutputChannel.java | 34 -
.../nephele/io/library/DirectoryReader.java | 103 ---
.../nephele/io/library/DirectoryWriter.java | 67 --
.../nephele/io/library/FileLineReader.java | 76 --
.../nephele/io/library/FileLineWriter.java | 75 --
.../eu/stratosphere/nephele/ipc/Client.java | 43 +-
.../nephele/jobgraph/AbstractJobVertex.java | 3 +-
.../nephele/jobgraph/DistributionPattern.java | 33 +
.../stratosphere/nephele/jobgraph/JobEdge.java | 3 +-
.../eu/stratosphere/nephele/jobgraph/JobID.java | 69 +-
.../nephele/jobgraph/JobVertexID.java | 2 +-
.../nephele/jobmanager/JobManager.java | 147 +---
.../jobmanager/scheduler/AbstractScheduler.java | 2 +-
.../jobmanager/scheduler/RecoveryLogic.java | 3 +-
.../nephele/managementgraph/ManagementEdge.java | 2 +-
.../managementgraph/ManagementEdgeID.java | 4 +-
.../managementgraph/ManagementGateID.java | 2 +-
.../managementgraph/ManagementGraph.java | 2 +-
.../managementgraph/ManagementGroupEdge.java | 2 +-
.../managementgraph/ManagementGroupVertex.java | 2 +-
.../ManagementGroupVertexID.java | 2 +-
.../managementgraph/ManagementVertexID.java | 2 +-
.../nephele/multicast/MulticastCluster.java | 210 -----
.../multicast/MulticastForwardingTable.java | 49 --
.../nephele/multicast/MulticastManager.java | 463 -----------
.../multicast/TopologyInformationSupplier.java | 30 -
.../nephele/multicast/TreeNode.java | 246 ------
.../nephele/profiling/TaskManagerProfiler.java | 5 +-
.../profiling/impl/TaskManagerProfilerImpl.java | 31 +-
.../protocols/ChannelLookupProtocol.java | 4 +-
.../protocols/ExtendedManagementProtocol.java | 13 -
.../protocols/TaskOperationProtocol.java | 14 +-
.../services/iomanager/BlockChannelAccess.java | 4 +-
.../services/memorymanager/MemoryManager.java | 3 +-
.../memorymanager/spi/DefaultMemoryManager.java | 17 +-
.../taskmanager/ExecutorThreadFactory.java | 35 +
.../stratosphere/nephele/taskmanager/Task.java | 257 +++++-
.../nephele/taskmanager/TaskKillResult.java | 44 -
.../nephele/taskmanager/TaskManager.java | 56 +-
.../BufferAvailabilityListener.java | 28 -
.../bufferprovider/BufferProvider.java | 82 --
.../bufferprovider/BufferProviderBroker.java | 24 -
.../bufferprovider/GlobalBufferPool.java | 135 ---
.../bufferprovider/LocalBufferPool.java | 287 -------
.../bufferprovider/LocalBufferPoolOwner.java | 54 --
.../AbstractOutputChannelContext.java | 84 --
.../AbstractOutputChannelForwarder.java | 109 ---
.../ByteBufferedChannelManager.java | 816 -------------------
.../bytebuffered/CanceledChannelSet.java | 211 -----
.../bytebuffered/ChannelContext.java | 36 -
.../ConnectionInfoLookupResponse.java | 176 ----
.../taskmanager/bytebuffered/GateContext.java | 21 -
.../bytebuffered/IncomingConnection.java | 125 ---
.../bytebuffered/IncomingConnectionThread.java | 223 -----
.../bytebuffered/InputChannelContext.java | 21 -
.../bytebuffered/InputGateContext.java | 24 -
.../InsufficientResourcesException.java | 37 -
.../bytebuffered/NetworkConnectionManager.java | 173 ----
.../bytebuffered/OutgoingConnection.java | 531 ------------
.../bytebuffered/OutgoingConnectionThread.java | 270 ------
.../bytebuffered/OutputChannelContext.java | 17 -
.../OutputChannelForwardingChain.java | 84 --
.../bytebuffered/OutputGateContext.java | 22 -
.../bytebuffered/ReceiverNotFoundEvent.java | 169 ----
.../bytebuffered/RemoteReceiver.java | 157 ----
.../bytebuffered/SenderHintEvent.java | 119 ---
.../taskmanager/bytebuffered/TaskContext.java | 24 -
.../bytebuffered/UnexpectedEnvelopeEvent.java | 81 --
.../runtime/ExecutorThreadFactory.java | 35 -
.../taskmanager/runtime/ForwardingBarrier.java | 75 --
.../taskmanager/runtime/RuntimeDispatcher.java | 38 -
.../runtime/RuntimeInputChannelContext.java | 303 -------
.../runtime/RuntimeInputGateContext.java | 183 -----
.../runtime/RuntimeOutputChannelBroker.java | 206 -----
.../runtime/RuntimeOutputChannelContext.java | 76 --
.../runtime/RuntimeOutputGateContext.java | 159 ----
.../taskmanager/runtime/RuntimeTask.java | 346 --------
.../taskmanager/runtime/RuntimeTaskContext.java | 211 -----
.../transferenvelope/AbstractDeserializer.java | 355 --------
.../transferenvelope/AbstractSerializer.java | 274 -------
.../CapacityConstrainedArrayQueue.java | 322 --------
.../transferenvelope/DefaultDeserializer.java | 94 ---
.../transferenvelope/DefaultSerializer.java | 38 -
.../NoBufferAvailableException.java | 53 --
.../transferenvelope/TransferEnvelope.java | 165 ----
.../TransferEnvelopeDispatcher.java | 37 -
.../TransferEnvelopeReceiverList.java | 86 --
.../nephele/util/BufferPoolConnector.java | 45 +
.../eu/stratosphere/nephele/util/TaskUtils.java | 42 -
.../runtime/iterative/io/FakeOutputTask.java | 2 +-
.../iterative/task/IterationHeadPactTask.java | 16 +-
.../task/IterationIntermediatePactTask.java | 7 +-
.../task/IterationSynchronizationSinkTask.java | 2 +-
.../iterative/task/IterationTailPactTask.java | 1 +
.../pact/runtime/shipping/OutputCollector.java | 31 +-
.../pact/runtime/shipping/OutputEmitter.java | 2 +-
.../runtime/shipping/RecordOutputCollector.java | 48 +-
.../runtime/shipping/RecordOutputEmitter.java | 2 +
.../pact/runtime/task/DataSinkTask.java | 19 +-
.../pact/runtime/task/DataSourceTask.java | 24 +-
.../pact/runtime/task/RegularPactTask.java | 250 +++---
.../ExceptionInChainedStubException.java | 8 +
.../SynchronousChainedCombineDriver.java | 2 +
.../pact/runtime/task/util/ReaderIterator.java | 2 +-
.../runtime/task/util/RecordReaderIterator.java | 2 +-
.../java/eu/stratosphere/runtime/io/Buffer.java | 93 +++
.../stratosphere/runtime/io/BufferRecycler.java | 26 +
.../runtime/io/api/AbstractRecordReader.java | 98 +++
.../io/api/AbstractSingleGateRecordReader.java | 69 ++
.../io/api/AbstractUnionRecordReader.java | 155 ++++
.../runtime/io/api/BufferWriter.java | 67 ++
.../runtime/io/api/ChannelSelector.java | 39 +
.../runtime/io/api/MutableReader.java | 32 +
.../runtime/io/api/MutableRecordReader.java | 120 +++
.../io/api/MutableUnionRecordReader.java | 37 +
.../eu/stratosphere/runtime/io/api/Reader.java | 30 +
.../stratosphere/runtime/io/api/ReaderBase.java | 67 ++
.../runtime/io/api/RecordReader.java | 154 ++++
.../runtime/io/api/RecordWriter.java | 151 ++++
.../io/api/RoundRobinChannelSelector.java | 47 ++
.../runtime/io/api/UnionRecordReader.java | 67 ++
.../runtime/io/channels/BufferOrEvent.java | 52 ++
.../runtime/io/channels/Channel.java | 97 +++
.../runtime/io/channels/ChannelCloseEvent.java | 33 +
.../runtime/io/channels/ChannelID.java | 39 +
.../runtime/io/channels/ChannelType.java | 26 +
.../io/channels/EndOfSuperstepEvent.java | 34 +
.../runtime/io/channels/InputChannel.java | 493 +++++++++++
.../runtime/io/channels/OutputChannel.java | 193 +++++
.../ReceiverAlreadyClosedException.java | 22 +
.../eu/stratosphere/runtime/io/gates/Gate.java | 174 ++++
.../stratosphere/runtime/io/gates/GateID.java | 24 +
.../runtime/io/gates/InputChannelResult.java | 23 +
.../runtime/io/gates/InputGate.java | 384 +++++++++
.../runtime/io/gates/OutputGate.java | 165 ++++
.../io/gates/RecordAvailabilityListener.java | 36 +
.../runtime/io/network/ChannelManager.java | 646 +++++++++++++++
.../network/ConnectionInfoLookupResponse.java | 143 ++++
.../network/InsufficientResourcesException.java | 37 +
.../LocalReceiverCancelledException.java | 37 +
.../io/network/NetworkConnectionManager.java | 176 ++++
.../runtime/io/network/RemoteReceiver.java | 157 ++++
.../runtime/io/network/SenderHintEvent.java | 117 +++
.../BufferAvailabilityListener.java | 28 +
.../network/bufferprovider/BufferProvider.java | 69 ++
.../bufferprovider/BufferProviderBroker.java | 24 +
.../bufferprovider/GlobalBufferPool.java | 123 +++
.../network/bufferprovider/LocalBufferPool.java | 306 +++++++
.../bufferprovider/LocalBufferPoolOwner.java | 56 ++
.../bufferprovider/SerialSingleBufferPool.java | 77 ++
.../runtime/io/network/envelope/Envelope.java | 169 ++++
.../io/network/envelope/EnvelopeDispatcher.java | 46 ++
.../io/network/envelope/EnvelopeReader.java | 212 +++++
.../network/envelope/EnvelopeReceiverList.java | 75 ++
.../io/network/envelope/EnvelopeWriter.java | 134 +++
.../envelope/NoBufferAvailableException.java | 53 ++
.../io/network/tcp/IncomingConnection.java | 115 +++
.../network/tcp/IncomingConnectionThread.java | 226 +++++
.../io/network/tcp/OutgoingConnection.java | 529 ++++++++++++
.../network/tcp/OutgoingConnectionThread.java | 276 +++++++
.../AdaptiveSpanningRecordDeserializer.java | 521 ++++++++++++
.../io/serialization/DataInputDeserializer.java | 307 +++++++
.../io/serialization/DataOutputSerializer.java | 259 ++++++
.../io/serialization/RecordDeserializer.java | 56 ++
.../io/serialization/RecordSerializer.java | 60 ++
.../serialization/SpanningRecordSerializer.java | 153 ++++
.../eu/stratosphere/nephele/AbstractIDTest.java | 62 ++
.../ChannelDeploymentDescriptorTest.java | 2 +-
.../GateDeploymentDescriptorTest.java | 12 +-
.../TaskDeploymentDescriptorTest.java | 2 +-
.../executiongraph/ExecutionGraphTest.java | 30 +-
.../ForwardTask1Input1Output.java | 9 +-
.../ForwardTask1Input2Outputs.java | 14 +-
.../ForwardTask2Inputs1Output.java | 9 +-
.../executiongraph/SelfCrossForwardTask.java | 6 +-
.../executiongraph/SelfCrossInputTask.java | 6 +-
.../stratosphere/nephele/fs/LineReaderTest.java | 79 --
.../nephele/fs/s3/S3FileSystemTest.java | 461 -----------
.../stratosphere/nephele/io/AbstractIDTest.java | 62 --
.../nephele/io/DefaultChannelSelectorTest.java | 46 --
.../nephele/io/channels/BufferTest.java | 192 -----
.../nephele/io/channels/MemoryBufferTest.java | 155 ----
.../io/channels/serialization/BooleanType.java | 77 --
.../channels/serialization/ByteArrayType.java | 88 --
.../serialization/ByteSubArrayType.java | 98 ---
.../io/channels/serialization/ByteType.java | 77 --
.../io/channels/serialization/CharType.java | 78 --
.../serialization/DeSerializerTest.java | 299 -------
.../io/channels/serialization/DoubleType.java | 78 --
.../io/channels/serialization/FloatType.java | 77 --
.../io/channels/serialization/IntType.java | 77 --
.../io/channels/serialization/LongType.java | 77 --
.../serialization/SerializationTestType.java | 52 --
.../io/channels/serialization/ShortType.java | 77 --
.../channels/serialization/UTFStringType.java | 86 --
.../serialization/UnsignedByteType.java | 77 --
.../serialization/UnsignedShortType.java | 77 --
.../io/library/FileLineReadWriteTest.java | 134 ---
.../nephele/jobmanager/DoubleSourceTask.java | 11 +-
.../nephele/jobmanager/DoubleTargetTask.java | 10 +-
.../nephele/jobmanager/ExceptionTask.java | 6 +-
.../nephele/jobmanager/ForwardTask.java | 10 +-
.../nephele/jobmanager/JobManagerITCase.java | 76 +-
.../nephele/jobmanager/UnionTask.java | 12 +-
.../scheduler/queue/QueueSchedulerTest.java | 10 +-
.../managementgraph/ManagementGraphTest.java | 6 +-
.../profiling/impl/InstanceProfilerTest.java | 2 +-
.../DefaultDeserializerTest.java | 358 --------
.../transferenvelope/DefaultSerializerTest.java | 313 -------
.../nephele/util/BufferPoolConnector.java | 53 --
.../nephele/util/DiscardingRecycler.java | 24 +
.../nephele/util/FileLineReader.java | 80 ++
.../nephele/util/FileLineWriter.java | 75 ++
.../nephele/util/TestBufferProvider.java | 76 ++
.../pact/runtime/task/DataSinkTaskTest.java | 36 +-
.../pact/runtime/task/DataSourceTaskTest.java | 28 +-
.../runtime/task/chaining/ChainTaskTest.java | 11 +-
.../runtime/task/util/OutputEmitterTest.java | 1 +
.../task/util/RecordOutputEmitterTest.java | 2 +
.../pact/runtime/test/util/MockEnvironment.java | 169 ++--
.../pact/runtime/test/util/TaskTestBase.java | 4 +-
.../stratosphere/runtime/fs/LineReaderTest.java | 78 ++
.../runtime/fs/s3/S3FileSystemTest.java | 460 +++++++++++
.../runtime/io/DefaultChannelSelectorTest.java | 47 ++
.../io/library/FileLineReadWriteTest.java | 136 ++++
.../envelope/EnvelopeReaderWriterTest.java | 394 +++++++++
.../DataInputOutputSerializerTest.java | 115 +++
.../io/serialization/PagedViewsTest.java | 160 ++++
.../SpanningRecordSerializationTest.java | 164 ++++
.../SpanningRecordSerializerTest.java | 219 +++++
.../io/serialization/types/AsciiStringType.java | 77 ++
.../io/serialization/types/BooleanType.java | 67 ++
.../io/serialization/types/ByteArrayType.java | 76 ++
.../serialization/types/ByteSubArrayType.java | 91 +++
.../io/serialization/types/ByteType.java | 67 ++
.../io/serialization/types/CharType.java | 68 ++
.../io/serialization/types/DoubleType.java | 68 ++
.../io/serialization/types/FloatType.java | 67 ++
.../runtime/io/serialization/types/IntType.java | 67 ++
.../io/serialization/types/LongType.java | 67 ++
.../types/SerializationTestType.java | 26 +
.../types/SerializationTestTypeFactory.java | 40 +
.../io/serialization/types/ShortType.java | 67 ++
.../serialization/types/UnsignedByteType.java | 67 ++
.../serialization/types/UnsignedShortType.java | 67 ++
.../runtime/io/serialization/types/Util.java | 90 ++
.../BroadcastVarsNepheleITCase.java | 4 +-
.../KMeansIterativeNepheleITCase.java | 3 +-
.../test/iterative/nephele/JobGraphUtils.java | 4 +-
.../CustomCompensatableDanglingPageRank.java | 10 +-
...mpensatableDanglingPageRankWithCombiner.java | 10 +-
.../CompensatableDanglingPageRank.java | 10 +-
.../query1Util/LineItemFilterTest.java | 6 +-
357 files changed, 12871 insertions(+), 20521 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-addons/spargel/src/main/java/eu/stratosphere/spargel/java/record/SpargelIteration.java
----------------------------------------------------------------------
diff --git a/stratosphere-addons/spargel/src/main/java/eu/stratosphere/spargel/java/record/SpargelIteration.java b/stratosphere-addons/spargel/src/main/java/eu/stratosphere/spargel/java/record/SpargelIteration.java
index 74bcaf3..14dcb35 100644
--- a/stratosphere-addons/spargel/src/main/java/eu/stratosphere/spargel/java/record/SpargelIteration.java
+++ b/stratosphere-addons/spargel/src/main/java/eu/stratosphere/spargel/java/record/SpargelIteration.java
@@ -155,6 +155,7 @@ public class SpargelIteration {
@Override
public void coGroup(Iterator<Record> messages, Iterator<Record> vertex, Collector<Record> out) throws Exception {
+
if (vertex.hasNext()) {
Record first = vertex.next();
first.getFieldInto(0, vertexKey);
@@ -270,4 +271,4 @@ public class SpargelIteration {
this.messagingFunction.postSuperstep();
}
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
index 80ae308..53b4cc1 100644
--- a/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
+++ b/stratosphere-compiler/src/main/java/eu/stratosphere/compiler/plantranslate/NepheleJobGraphGenerator.java
@@ -52,8 +52,9 @@ import eu.stratosphere.compiler.plan.WorksetPlanNode;
import eu.stratosphere.configuration.ConfigConstants;
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.configuration.ConfigConstants;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.nephele.jobgraph.AbstractJobOutputVertex;
import eu.stratosphere.nephele.jobgraph.AbstractJobVertex;
import eu.stratosphere.nephele.jobgraph.JobGraph;
@@ -1224,7 +1225,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
// connect the fake tail
try {
- rootOfStepFunctionVertex.connectTo(fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ rootOfStepFunctionVertex.connectTo(fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
} catch (JobGraphDefinitionException e) {
throw new CompilerException("Bug: Cannot connect iteration tail vertex fake tail task");
}
@@ -1267,7 +1268,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
// connect the fake tail
try {
- rootOfTerminationCriterionVertex.connectTo(fakeTailTerminationCriterion, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ rootOfTerminationCriterionVertex.connectTo(fakeTailTerminationCriterion, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
} catch (JobGraphDefinitionException e) {
throw new CompilerException("Bug: Cannot connect iteration tail vertex fake tail task for termination criterion");
}
@@ -1401,7 +1402,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
// connect the fake tail
try {
- nextWorksetVertex.connectTo(fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ nextWorksetVertex.connectTo(fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
} catch (JobGraphDefinitionException e) {
throw new CompilerException("Bug: Cannot connect iteration tail vertex fake tail task");
}
@@ -1440,7 +1441,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
// connect the fake tail
try {
- solutionDeltaVertex.connectTo(fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ solutionDeltaVertex.connectTo(fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
} catch (JobGraphDefinitionException e) {
throw new CompilerException("Bug: Cannot connect iteration tail vertex fake tail task");
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java b/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
index 51c0a2f..3b9ba3d 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/configuration/ConfigConstants.java
@@ -344,12 +344,6 @@ public final class ConfigConstants {
*/
public static final int DEFAULT_FS_STREAM_OPENING_TIMEOUT = 0;
- /**
- * The config parameter defining whether to use the special multicast logic
- * for broadcasts.
- */
- public static final boolean DEFAULT_USE_MULTICAST_FOR_BROADCAST = false;
-
// ------------------------ File System Bahavior ------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-core/src/main/java/eu/stratosphere/core/memory/MemorySegment.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/core/memory/MemorySegment.java b/stratosphere-core/src/main/java/eu/stratosphere/core/memory/MemorySegment.java
index 2698992..2d63fa4 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/core/memory/MemorySegment.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/core/memory/MemorySegment.java
@@ -103,6 +103,11 @@ public class MemorySegment {
public final boolean isFreed() {
return this.memory == null;
}
+
+ public final void free() {
+ this.wrapper = null;
+ this.memory = null;
+ }
/**
* Gets the size of the memory segment, in bytes. Because segments
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java b/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
index 73671fa..5b06547 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
@@ -70,7 +70,7 @@ public final class Record implements Value, CopyableValue<Record> {
private Value[] writeFields; // the cache for objects into which the binary representations are read
- private int binaryLen; // the length of the contents in the binary buffer that is valid
+ public int binaryLen; // the length of the contents in the binary buffer that is valid
private int numFields; // the number of fields in the record
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java
new file mode 100644
index 0000000..476e22a
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java
@@ -0,0 +1,177 @@
+/***********************************************************************************************************************
+ * 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.nephele;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.util.StringUtils;
+
+/**
+ * A statistically unique identification number.
+ */
+public class AbstractID implements IOReadableWritable {
+
+ /** The size of a long in bytes */
+ private static final int SIZE_OF_LONG = 8;
+
+ /** The size of the ID in byte */
+ public static final int SIZE = 2 * SIZE_OF_LONG;
+
+ /** The upper part of the actual ID */
+ private long upperPart;
+
+ /** The lower part of the actual ID */
+ private long lowerPart;
+
+ /**
+ * Constructs a new ID with a specific bytes value.
+ */
+ public AbstractID(byte[] bytes) {
+ if (bytes.length != SIZE) {
+ throw new IllegalArgumentException("Argument bytes must by an array of " + SIZE + " bytes");
+ }
+
+ this.lowerPart = byteArrayToLong(bytes, 0);
+ this.upperPart = byteArrayToLong(bytes, SIZE_OF_LONG);
+ }
+
+ /**
+ * Constructs a new abstract ID.
+ *
+ * @param lowerPart the lower bytes of the ID
+ * @param upperPart the higher bytes of the ID
+ */
+ public AbstractID(long lowerPart, long upperPart) {
+ this.lowerPart = lowerPart;
+ this.upperPart = upperPart;
+ }
+
+ /**
+ * Creates a new abstract ID from the given one.
+ * <p>
+ * The given and the newly created abstract ID will be identical, i.e. a comparison by <code>equals</code> will
+ * return <code>true</code> and both objects will have the same hash code.
+ *
+ * @param id the abstract ID to copy
+ */
+ public AbstractID(AbstractID id) {
+ this.lowerPart = id.lowerPart;
+ this.upperPart = id.upperPart;
+ }
+
+ /**
+ * Constructs a new random ID from a uniform distribution.
+ */
+ public AbstractID() {
+ this.lowerPart = generateRandomLong();
+ this.upperPart = generateRandomLong();
+ }
+
+ /**
+ * Generates a uniformly distributed random positive long.
+ *
+ * @return a uniformly distributed random positive long
+ */
+ protected static long generateRandomLong() {
+ return (long) (Math.random() * Long.MAX_VALUE);
+ }
+
+ /**
+ * Converts the given byte array to a long.
+ *
+ * @param ba the byte array to be converted
+ * @param offset the offset indicating at which byte inside the array the conversion shall begin
+ * @return the long variable
+ */
+ private static long byteArrayToLong(byte[] ba, int offset) {
+ long l = 0;
+
+ for (int i = 0; i < SIZE_OF_LONG; ++i) {
+ l |= (ba[offset + SIZE_OF_LONG - 1 - i] & 0xffL) << (i << 3);
+ }
+
+ return l;
+ }
+
+ /**
+ * Converts a long to a byte array.
+ *
+ * @param l the long variable to be converted
+ * @param ba the byte array to store the result the of the conversion
+ * @param offset offset indicating at what position inside the byte array the result of the conversion shall be stored
+ */
+ private static void longToByteArray(final long l, final byte[] ba, final int offset) {
+ for (int i = 0; i < SIZE_OF_LONG; ++i) {
+ final int shift = i << 3; // i * 8
+ ba[offset + SIZE_OF_LONG - 1 - i] = (byte) ((l & (0xffL << shift)) >>> shift);
+ }
+ }
+
+ /**
+ * Sets an ID from another ID by copying its internal byte representation.
+ *
+ * @param src source ID
+ */
+ public void setID(AbstractID src) {
+ this.lowerPart = src.lowerPart;
+ this.upperPart = src.upperPart;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj != null && obj instanceof AbstractID) {
+ AbstractID src = (AbstractID) obj;
+ return src.lowerPart == this.lowerPart && src.upperPart == this.upperPart;
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return ((int) this.lowerPart) ^
+ ((int) (this.lowerPart >>> 32)) ^
+ ((int) this.upperPart) ^
+ ((int) (this.upperPart >>> 32));
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ this.lowerPart = in.readLong();
+ this.upperPart = in.readLong();
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeLong(this.lowerPart);
+ out.writeLong(this.upperPart);
+ }
+
+ public void write(ByteBuffer buffer) {
+ buffer.putLong(this.lowerPart);
+ buffer.putLong(this.upperPart);
+ }
+
+ @Override
+ public String toString() {
+ final byte[] ba = new byte[SIZE];
+ longToByteArray(this.lowerPart, ba, 0);
+ longToByteArray(this.upperPart, ba, SIZE_OF_LONG);
+ return StringUtils.byteToHexString(ba);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/ForceCheckpoint.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/ForceCheckpoint.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/ForceCheckpoint.java
deleted file mode 100644
index 012dc02..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/ForceCheckpoint.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/***********************************************************************************************************************
- * 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.annotations;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-/**
- * Annotation to force a checkpoint decision.
- *
- */
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.TYPE)
-public @interface ForceCheckpoint {
-
- boolean checkpoint();
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateful.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateful.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateful.java
deleted file mode 100644
index 433478b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateful.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/***********************************************************************************************************************
- * 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.annotations;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-/**
- * This annotation can be used to indicate a stateful task.
- *
- */
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.TYPE)
-public @interface Stateful {
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java
deleted file mode 100644
index b3d5e38..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/Stateless.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/***********************************************************************************************************************
- * 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.annotations;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-/**
- * This annotation can be used to indicate a stateless task.
- *
- */
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.TYPE)
-public @interface Stateless {
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java
deleted file mode 100644
index 317b112..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/annotations/TaskAnnotation.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/***********************************************************************************************************************
- * 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.annotations;
-
-import java.lang.annotation.ElementType;
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-import java.lang.annotation.Target;
-
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.CONSTRUCTOR)
-public @interface TaskAnnotation {
-
- boolean statefull();
- double selektivity() default 1;
-
-}
-
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptor.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptor.java
index c39aad0..4068e5b 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptor.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptor.java
@@ -18,7 +18,7 @@ import java.io.DataOutput;
import java.io.IOException;
import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
/**
* A channel deployment descriptor contains all the information necessary to deploy either an input or an output channel
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptor.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptor.java
index 8a78154..02d6578 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptor.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptor.java
@@ -21,8 +21,8 @@ import java.util.Iterator;
import java.util.List;
import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.gates.GateID;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.nephele.util.EnumUtils;
/**
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventExample.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventExample.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventExample.java
deleted file mode 100644
index 4e6b2e4..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventExample.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/***********************************************************************************************************************
- * 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.
- **********************************************************************************************************************/
-
-/*
- * Copyright 2010 casp.
- *
- * 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.
- * under the License.
- */
-package eu.stratosphere.nephele.example.events;
-
-import java.io.IOException;
-
-import eu.stratosphere.configuration.ConfigConstants;
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.client.JobClient;
-import eu.stratosphere.nephele.client.JobSubmissionResult;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.library.FileLineReader;
-import eu.stratosphere.nephele.io.library.FileLineWriter;
-import eu.stratosphere.nephele.jobgraph.JobFileInputVertex;
-import eu.stratosphere.nephele.jobgraph.JobFileOutputVertex;
-import eu.stratosphere.nephele.jobgraph.JobGraph;
-import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
-import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
-
-/**
- */
-public class EventExample {
-
- public static void main(String[] args) {
-
- JobGraph jobGraph = new JobGraph("Grep Example Job");
-
- JobFileInputVertex input = new JobFileInputVertex("Input 1", jobGraph);
- input.setFileInputClass(FileLineReader.class);
- input.setFilePath(new Path("file:///Users/casp/test2.txt"));
-
- JobTaskVertex task1 = new JobTaskVertex("Task 1", jobGraph);
- task1.setTaskClass(EventSender.class);
-
- JobTaskVertex task2 = new JobTaskVertex("Task 2", jobGraph);
- task2.setTaskClass(EventReceiver.class);
-
- JobFileOutputVertex output = new JobFileOutputVertex("Output 1", jobGraph);
- output.setFileOutputClass(FileLineWriter.class);
- output.setFilePath(new Path("file:///Users/casp/output.txt"));
-
- jobGraph.addJar(new Path("file:///Users/casp/EventTask.jar"));
- jobGraph.addJar(new Path("file:///Users/casp/StringTaskEvent.jar"));
- try {
-
- input.connectTo(task1, ChannelType.INMEMORY);
- task1.connectTo(task2, ChannelType.INMEMORY);
- task2.connectTo(output, ChannelType.INMEMORY);
-
- } catch (JobGraphDefinitionException e) {
- e.printStackTrace();
- }
-
- Configuration conf = new Configuration();
- conf.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "127.0.0.1");
- conf.setString(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, "6023");
-
- try {
- JobClient jobClient = new JobClient(jobGraph, conf);
- JobSubmissionResult result = jobClient.submitJob();
- System.out.println(result.getDescription());
- } catch (IOException ioe) {
- ioe.printStackTrace();
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventReceiver.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventReceiver.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventReceiver.java
deleted file mode 100644
index 3e5d322..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventReceiver.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/***********************************************************************************************************************
- * 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.example.events;
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.event.task.StringTaskEvent;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-/**
- */
-public class EventReceiver extends AbstractTask {
-
- // this is just a dummy input gate...
- private RecordReader<StringRecord> input = null;
-
- private RecordWriter<StringRecord> output = null;
-
- @Override
- public void registerInputOutput() {
- this.input = new RecordReader<StringRecord>(this, StringRecord.class);
- this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
- this.input.subscribeToEvent(new MyEventListener(), StringTaskEvent.class);
- }
-
- @Override
- public void invoke() throws Exception {
-
- while (this.input.hasNext()) {
-
- StringRecord s = input.next();
- this.output.emit(s);
- }
-
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventSender.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventSender.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventSender.java
deleted file mode 100644
index 4961dac..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/EventSender.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/***********************************************************************************************************************
- * 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.example.events;
-
-/*
- * Copyright 2010 casp.
- *
- * 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.
- * under the License.
- */
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.event.task.StringTaskEvent;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-/**
- */
-public class EventSender extends AbstractTask {
-
- // this is just a dummy output gate...
- private RecordReader<StringRecord> input = null;
-
- private RecordWriter<StringRecord> output = null;
-
- @Override
- public void registerInputOutput() {
- this.input = new RecordReader<StringRecord>(this, StringRecord.class);
- this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
- }
-
- @Override
- public void invoke() throws Exception {
-
- int i = 1;
- while (this.input.hasNext()) {
- i++;
- if (i % 1000 == 0) {
- this.output.publishEvent(new StringTaskEvent("this is the " + i + "th message"));
- }
- StringRecord s = input.next();
- this.output.emit(s);
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/MyEventListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/MyEventListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/MyEventListener.java
deleted file mode 100644
index f285d44..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/events/MyEventListener.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/***********************************************************************************************************************
- * 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.
- **********************************************************************************************************************/
-
-/*
- * Copyright 2010 casp.
- *
- * 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.
- * under the License.
- */
-package eu.stratosphere.nephele.example.events;
-
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.event.task.EventListener;
-import eu.stratosphere.nephele.event.task.StringTaskEvent;
-
-/**
- */
-public class MyEventListener implements EventListener {
-
- @Override
- public void eventOccurred(AbstractTaskEvent event) {
- if (event instanceof StringTaskEvent) {
- System.out.println("Message Event received: " + ((StringTaskEvent) event).getString());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/Grep.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/Grep.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/Grep.java
deleted file mode 100644
index 9afef9b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/Grep.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/***********************************************************************************************************************
- * 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.example.grep;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.client.JobClient;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.library.FileLineReader;
-import eu.stratosphere.nephele.io.library.FileLineWriter;
-import eu.stratosphere.nephele.jobgraph.JobFileInputVertex;
-import eu.stratosphere.nephele.jobgraph.JobFileOutputVertex;
-import eu.stratosphere.nephele.jobgraph.JobGraph;
-import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
-import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
-import eu.stratosphere.nephele.util.JarFileCreator;
-
-public class Grep {
-
- public static void main(String[] args) {
-
- JobGraph jobGraph = new JobGraph("Grep Example Job");
-
- JobFileInputVertex input = new JobFileInputVertex("Input 1", jobGraph);
- input.setFileInputClass(FileLineReader.class);
- input.setFilePath(new Path("file:///home/ec2-user/test.txt"));
- input.setInstanceType("t1.micro");
-
- JobTaskVertex task1 = new JobTaskVertex("Task 1", jobGraph);
- task1.setTaskClass(GrepTask.class);
- task1.setInstanceType("t1.micro");
-
-
- JobFileOutputVertex output = new JobFileOutputVertex("Output 1", jobGraph);
- output.setFileOutputClass(FileLineWriter.class);
- output.setFilePath(new Path("file:///tmp/"));
- output.setInstanceType("t1.micro");
-
- try {
-
- input.connectTo(task1, ChannelType.INMEMORY);
- task1.connectTo(output, ChannelType.INMEMORY);
-
- } catch (JobGraphDefinitionException e) {
- e.printStackTrace();
- }
-
- // Create jar file and attach it
- final File jarFile = new File("/tmp/grepJob.jar");
- final JarFileCreator jarFileCreator = new JarFileCreator(jarFile);
- jarFileCreator.addClass(GrepTask.class);
-
- try {
- jarFileCreator.createJarFile();
- System.out.println("done creating!!");
- } catch (IOException ioe) {
-
- if (jarFile.exists()) {
- jarFile.delete();
- }
-
- System.out.println("ERROR creating jar");
- return;
- }
-
- jobGraph.addJar(new Path("file://" + jarFile.getAbsolutePath()));
-
- // Submit job
- Configuration conf = new Configuration();
-
- jobGraph.getJobConfiguration().setString("job.cloud.awsaccessid", "xxx");
- jobGraph.getJobConfiguration().setString("job.cloud.awssecretkey", "xxx");
- jobGraph.getJobConfiguration().setString("job.cloud.sshkeypair", "caspeu");
- jobGraph.getJobConfiguration().setString("job.ec2.image.id", "ami-d64474a2");
-
- InetSocketAddress jobmanager = new InetSocketAddress("127.0.0.1", 6123);
-
-
- try {
- final JobClient jobClient = new JobClient(jobGraph, conf, jobmanager);
- System.out.println("submitting");
- jobClient.submitJobAndWait();
- System.out.println("done.");
- } catch (Exception e) {
- System.out.println(e);
- }
-
- if (jarFile.exists()) {
- jarFile.delete();
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/GrepTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/GrepTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/GrepTask.java
deleted file mode 100644
index ae72f1c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/grep/GrepTask.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/***********************************************************************************************************************
- * 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.example.grep;
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-public class GrepTask extends AbstractTask {
-
- private RecordReader<StringRecord> input = null;
-
- private RecordWriter<StringRecord> output = null;
-
- private int i = 0;
-
- @Override
- public void invoke() throws Exception {
-
- while (this.input.hasNext()) {
-
- StringRecord s = input.next();
- this.output.emit(s);
- i++;
- }
-
- System.out.println("GREP: Emmited all " + i + " records");
- }
-
- @Override
- public void registerInputOutput() {
- this.input = new RecordReader<StringRecord>(this, StringRecord.class);
- this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTest.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTest.java
deleted file mode 100644
index f85024c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTest.java
+++ /dev/null
@@ -1,224 +0,0 @@
-/***********************************************************************************************************************
- * 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.example.speedtest;
-
-import java.io.File;
-import java.io.IOException;
-
-import eu.stratosphere.api.common.JobExecutionResult;
-import eu.stratosphere.configuration.ConfigConstants;
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.client.JobClient;
-import eu.stratosphere.nephele.client.JobExecutionException;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.jobgraph.JobGenericInputVertex;
-import eu.stratosphere.nephele.jobgraph.JobGenericOutputVertex;
-import eu.stratosphere.nephele.jobgraph.JobGraph;
-import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
-import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
-import eu.stratosphere.nephele.util.JarFileCreator;
-
-/**
- * This class implements a speed test for Nephele. It's primary purpose is to benchmark the performance of Nephele's
- * network channels with different degrees of parallelism.
- *
- */
-public final class SpeedTest {
-
- /**
- * Configuration key to specify the amount of data to be send in GB.
- */
- static final String DATA_VOLUME_CONFIG_KEY = "data.volume";
-
- /**
- * Entry point to the application.
- *
- * @param args
- * the provided arguments
- */
- public static void main(final String[] args) {
-
- // Parse the arguments first
- if (args.length < 4) {
- System.err
- .println("Insufficient number of arguments. Please provide <job manager address> <amount of data to send in GB> <number of subtasks> <number of subtasks per task> (<use forwarder>)");
- System.exit(1);
- return;
- }
-
- // Determine the job manager address
- final String jobManagerAddress = args[0];
-
- // Determine amount of data to send in GB
- int amountOfDataToSend = 0;
- try {
- amountOfDataToSend = Integer.parseInt(args[1]);
- } catch (NumberFormatException e) {
- System.err.println("Cannot parse amount of data to send. Please provide a positive integer value.");
- System.exit(1);
- return;
- }
-
- if (amountOfDataToSend <= 0 || amountOfDataToSend > 1024) {
- System.err
- .println("Please provide an integer value between 1 and 1024 indicating the amount of data to send in GB.");
- System.exit(1);
- return;
- }
-
- // Determine the number of subtasks
- int numberOfSubtasks = 0;
- try {
- numberOfSubtasks = Integer.parseInt(args[2]);
- } catch (NumberFormatException e) {
- System.err.println("Cannot parse the number of subtasks. Please provide a positive integer value.");
- System.exit(1);
- return;
- }
-
- if (numberOfSubtasks <= 0) {
- System.err.println("Please provide a positive integer value indicating the number of subtasks.");
- System.exit(1);
- return;
- }
-
- // Determine the number of subtasks per instance
- int numberOfSubtasksPerInstance = 0;
- try {
- numberOfSubtasksPerInstance = Integer.parseInt(args[3]);
- } catch (NumberFormatException e) {
- System.err
- .println("Cannot parse the number of subtasks per instance. Please provide a positive integer value.");
- System.exit(1);
- return;
- }
-
- if (numberOfSubtasksPerInstance <= 0) {
- System.err
- .println("Please provide a positive integer value indicating the number of subtasks per instance.");
- System.exit(1);
- return;
- }
-
- // Determine whether to use a forwarder or not
- boolean useForwarder = false;
- if (args.length >= 5) {
- useForwarder = Boolean.parseBoolean(args[4]);
- }
-
- final JobGraph jobGraph = new JobGraph("Nephele Speed Test");
-
- final JobGenericInputVertex producer = new JobGenericInputVertex("Speed Test Producer", jobGraph);
- producer.setInputClass(SpeedTestProducer.class);
- producer.setNumberOfSubtasks(numberOfSubtasks);
- producer.setNumberOfSubtasksPerInstance(numberOfSubtasksPerInstance);
- producer.getConfiguration().setInteger(DATA_VOLUME_CONFIG_KEY, amountOfDataToSend);
-
- JobTaskVertex forwarder = null;
- if (useForwarder) {
- forwarder = new JobTaskVertex("Speed Test Forwarder", jobGraph);
- forwarder.setTaskClass(SpeedTestForwarder.class);
- forwarder.setNumberOfSubtasks(numberOfSubtasks);
- forwarder.setNumberOfSubtasksPerInstance(numberOfSubtasksPerInstance);
- }
-
- final JobGenericOutputVertex consumer = new JobGenericOutputVertex("Speed Test Consumer", jobGraph);
- consumer.setOutputClass(SpeedTestConsumer.class);
- consumer.setNumberOfSubtasks(numberOfSubtasks);
- consumer.setNumberOfSubtasksPerInstance(numberOfSubtasksPerInstance);
-
- // Set vertex sharing
- producer.setVertexToShareInstancesWith(consumer);
- if (forwarder != null) {
- forwarder.setVertexToShareInstancesWith(consumer);
- }
-
- // Connect the vertices
- try {
- if (forwarder == null) {
- producer.connectTo(consumer, ChannelType.NETWORK,
- DistributionPattern.BIPARTITE);
- } else {
- producer.connectTo(forwarder, ChannelType.NETWORK,
- DistributionPattern.BIPARTITE);
- forwarder.connectTo(consumer, ChannelType.NETWORK,
- DistributionPattern.BIPARTITE);
- }
-
- } catch (JobGraphDefinitionException e) {
- e.printStackTrace();
- System.exit(1);
- return;
- }
-
- File jarFile = null;
-
- try {
-
- // Create jar file of job
- jarFile = File.createTempFile("speedtest", "jar");
- jarFile.deleteOnExit();
-
- final JarFileCreator jfc = new JarFileCreator(jarFile);
- jfc.addClass(SpeedTest.class);
- jfc.addClass(SpeedTestProducer.class);
- jfc.addClass(SpeedTestForwarder.class);
- jfc.addClass(SpeedTestConsumer.class);
- jfc.addClass(SpeedTestRecord.class);
- jfc.createJarFile();
-
- jobGraph.addJar(new Path("file://" + jarFile.getAbsolutePath()));
-
- final Configuration clientConfiguration = new Configuration();
- clientConfiguration.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, jobManagerAddress);
- clientConfiguration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
- ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT);
-
- final JobClient jobClient = new JobClient(jobGraph, clientConfiguration);
- final JobExecutionResult jobResult = jobClient.submitJobAndWait();
- final long executionTime = jobResult.getNetRuntime();
-
- // Calculate throughput in MBit/s and output it
- System.out.print("Job finished with a throughput of " + toMBitPerSecond(amountOfDataToSend, executionTime));
-
- } catch (IOException ioe) {
- ioe.printStackTrace();
- System.exit(1);
- return;
- } catch (JobExecutionException jee) {
- jee.printStackTrace();
- System.exit(1);
- return;
- }
- }
-
- /**
- * Calculates the throughput in MBit/s from the given amount of data that has been sent and the execution time.
- *
- * @param amountOfDataToSend
- * the amount of data that has been sent in GB
- * @param executionTime
- * the execution time in milliseconds
- * @return the resulting throughput in MBit/s
- */
- private static int toMBitPerSecond(final int amountOfDataToSend, final long executionTime) {
-
- final double dataVolumeInMBit = amountOfDataToSend * 8192.0;
- final double executionTimeInSeconds = executionTime / 1000.0;
-
- return (int) Math.round(dataVolumeInMBit / executionTimeInSeconds);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestConsumer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestConsumer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestConsumer.java
deleted file mode 100644
index eb40196..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestConsumer.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/***********************************************************************************************************************
- * 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.example.speedtest;
-
-import eu.stratosphere.nephele.io.MutableRecordReader;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
-
-/**
- * This class implements the consumer task of the speed test. The consumer task simply drops all received records.
- *
- */
-public class SpeedTestConsumer extends AbstractOutputTask {
-
- /**
- * The record reader used to read the incoming records.
- */
- private MutableRecordReader<SpeedTestRecord> input;
-
-
- @Override
- public void registerInputOutput() {
-
- this.input = new MutableRecordReader<SpeedTestRecord>(this);
- }
-
-
- @Override
- public void invoke() throws Exception {
-
- final SpeedTestRecord record = new SpeedTestRecord();
- while (this.input.next(record)) {
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestForwarder.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestForwarder.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestForwarder.java
deleted file mode 100644
index 91090a8..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestForwarder.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/***********************************************************************************************************************
- * 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.example.speedtest;
-
-import eu.stratosphere.nephele.io.MutableRecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-/**
- * This class implements a forwarder task for the speed test. The forwarder task immediately outputs every record it
- * reads.
- *
- */
-public final class SpeedTestForwarder extends AbstractTask {
-
- /**
- * The record reader used to read incoming records.
- */
- private MutableRecordReader<SpeedTestRecord> input;
-
- /**
- * The record writer used to forward incoming records.
- */
- private RecordWriter<SpeedTestRecord> output;
-
-
- @Override
- public void registerInputOutput() {
-
- this.input = new MutableRecordReader<SpeedTestRecord>(this);
- this.output = new RecordWriter<SpeedTestRecord>(this, SpeedTestRecord.class);
- }
-
-
- @Override
- public void invoke() throws Exception {
-
- final SpeedTestRecord record = new SpeedTestRecord();
- while (this.input.next(record)) {
- this.output.emit(record);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestProducer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestProducer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestProducer.java
deleted file mode 100644
index 78ea23d..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestProducer.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/***********************************************************************************************************************
- * 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.example.speedtest;
-
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractGenericInputTask;
-
-/**
- * This class implements the producer task which produces test records for the speed test.
- *
- */
-public final class SpeedTestProducer extends AbstractGenericInputTask {
-
- /**
- * The record writer to emit the produced records.
- */
- private RecordWriter<SpeedTestRecord> writer;
-
-
- @Override
- public void registerInputOutput() {
-
- this.writer = new RecordWriter<SpeedTestRecord>(this, SpeedTestRecord.class);
- }
-
-
- @Override
- public void invoke() throws Exception {
-
- // Determine the amount of data to send per subtask
- final int dataVolumePerSubtaskInMB = getTaskConfiguration().getInteger(SpeedTest.DATA_VOLUME_CONFIG_KEY, 1)
- * 1024 / getCurrentNumberOfSubtasks();
-
- final long numberOfRecordsToEmit = dataVolumePerSubtaskInMB * 1024 * 1024 / SpeedTestRecord.RECORD_SIZE;
-
- final SpeedTestRecord record = new SpeedTestRecord();
-
- for (long i = 0; i < numberOfRecordsToEmit; ++i) {
- this.writer.emit(record);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestRecord.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestRecord.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestRecord.java
deleted file mode 100644
index b3522f3..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/speedtest/SpeedTestRecord.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/***********************************************************************************************************************
- * 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.example.speedtest;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- * This class implements the record type used for the speed test.
- */
-public final class SpeedTestRecord implements IOReadableWritable {
-
- /**
- * The size of a single record in bytes.
- */
- static final int RECORD_SIZE = 128;
-
- /**
- * The byte buffer which actually stored the record's data.
- */
- private final byte[] buf = new byte[RECORD_SIZE];
-
- /**
- * Constructs a new record and initializes it.
- */
- public SpeedTestRecord() {
- for (int i = 0; i < RECORD_SIZE; ++i) {
- this.buf[i] = (byte) (i % 128);
- }
- }
-
- @Override
- public void write(final DataOutput out) throws IOException {
- out.write(this.buf);
- }
-
- @Override
- public void read(final DataInput in) throws IOException {
- in.readFully(this.buf);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ConsumerTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ConsumerTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ConsumerTask.java
deleted file mode 100644
index 209a81e..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ConsumerTask.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/***********************************************************************************************************************
- * 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.example.union;
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.template.AbstractFileOutputTask;
-
-public class ConsumerTask extends AbstractFileOutputTask {
-
- private RecordReader<StringRecord> input;
-
- @Override
- public void registerInputOutput() {
-
- this.input = new RecordReader<StringRecord>(this, StringRecord.class);
-
- }
-
- @Override
- public void invoke() throws Exception {
-
- int count = 0;
-
- while (this.input.hasNext()) {
- this.input.next();
- ++count;
- }
-
- System.out.println("Consumer receiver " + count + " records in total");
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ProducerTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ProducerTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ProducerTask.java
deleted file mode 100644
index 647a07b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/example/union/ProducerTask.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/***********************************************************************************************************************
- * 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.example.union;
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractFileInputTask;
-
-public class ProducerTask extends AbstractFileInputTask {
-
- private static final int NUMBER_OF_RECORDS_TO_PRODUCE = 1000000;
-
- private RecordWriter<StringRecord> output;
-
- @Override
- public void registerInputOutput() {
-
- this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
- }
-
- @Override
- public void invoke() throws Exception {
-
- for (int i = 0; i < NUMBER_OF_RECORDS_TO_PRODUCE; ++i) {
-
- final StringRecord record = new StringRecord("Record " + i + " of " + this);
- this.output.emit(record);
- }
-
- }
-
-}
[19/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
index 1ea5b1b..5240fc8 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskManager.java
@@ -65,7 +65,6 @@ import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
import eu.stratosphere.nephele.instance.HardwareDescription;
import eu.stratosphere.nephele.instance.HardwareDescriptionFactory;
import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.io.channels.ChannelID;
import eu.stratosphere.nephele.ipc.RPC;
import eu.stratosphere.nephele.ipc.Server;
import eu.stratosphere.nephele.jobgraph.JobID;
@@ -80,12 +79,11 @@ import eu.stratosphere.nephele.protocols.TaskOperationProtocol;
import eu.stratosphere.nephele.services.iomanager.IOManager;
import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
import eu.stratosphere.nephele.services.memorymanager.spi.DefaultMemoryManager;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ByteBufferedChannelManager;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.InsufficientResourcesException;
-import eu.stratosphere.nephele.taskmanager.runtime.ExecutorThreadFactory;
-import eu.stratosphere.nephele.taskmanager.runtime.RuntimeTask;
import eu.stratosphere.nephele.util.SerializableArrayList;
import eu.stratosphere.pact.runtime.cache.FileCache;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.network.ChannelManager;
+import eu.stratosphere.runtime.io.network.InsufficientResourcesException;
import eu.stratosphere.util.StringUtils;
/**
@@ -128,10 +126,10 @@ public class TaskManager implements TaskOperationProtocol {
private final InstanceConnectionInfo localInstanceConnectionInfo;
/**
- * The instance of the {@link ByteBufferedChannelManager} which is responsible for
+ * The instance of the {@link eu.stratosphere.nephele.taskmanager.io.bytebuffered.ChannelManager} which is responsible for
* setting up and cleaning up the byte buffered channels of the tasks.
*/
- private final ByteBufferedChannelManager byteBufferedChannelManager;
+ private final ChannelManager channelManager;
/**
* Instance of the task manager profile if profiling is enabled.
@@ -279,14 +277,24 @@ public class TaskManager implements TaskOperationProtocol {
final int pageSize = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE);
+ // Initialize network buffer pool
+ int numBuffers = GlobalConfiguration.getInteger(
+ ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
+ ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
+
+ int bufferSize = GlobalConfiguration.getInteger(
+ ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
+ ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE);
+
// Initialize the byte buffered channel manager
+ ChannelManager channelManager = null;
try {
- this.byteBufferedChannelManager = new ByteBufferedChannelManager(this.lookupService,
- this.localInstanceConnectionInfo);
- } catch (Exception e) {
- LOG.fatal("Cannot create byte channel manager:" + e.getMessage(), e);
- throw new Exception("Failed to instantiate Byte-buffered channel manager. " + e.getMessage(), e);
+ channelManager = new ChannelManager(this.lookupService, this.localInstanceConnectionInfo, numBuffers, bufferSize);
+ } catch (IOException ioe) {
+ LOG.error(StringUtils.stringifyException(ioe));
+ throw new Exception("Failed to instantiate Byte-buffered channel manager. " + ioe.getMessage(), ioe);
}
+ this.channelManager = channelManager;
{
HardwareDescription resources = HardwareDescriptionFactory.extractFromSystem();
@@ -598,7 +606,6 @@ public class TaskManager implements TaskOperationProtocol {
return new TaskCancelResult(id, AbstractTaskResult.ReturnCode.SUCCESS);
}
-
@Override
public TaskKillResult killTask(final ExecutionVertexID id) throws IOException {
@@ -661,13 +668,11 @@ public class TaskManager implements TaskOperationProtocol {
}
final Configuration jobConfiguration = tdd.getJobConfiguration();
- final Set<ChannelID> activeOutputChannels = null; // TODO: Fix me
// Register the task
Task task;
try {
- task = createAndRegisterTask(vertexID, jobConfiguration, re,
- activeOutputChannels);
+ task = createAndRegisterTask(vertexID, jobConfiguration, re);
} catch (InsufficientResourcesException e) {
final TaskSubmissionResult result = new TaskSubmissionResult(vertexID,
AbstractTaskResult.ReturnCode.INSUFFICIENT_RESOURCES);
@@ -707,12 +712,10 @@ public class TaskManager implements TaskOperationProtocol {
* the job configuration that has been attached to the original job graph
* @param environment
* the environment of the task to be registered
- * @param activeOutputChannels
- * the set of initially active output channels
* @return the task to be started or <code>null</code> if a task with the same ID was already running
*/
private Task createAndRegisterTask(final ExecutionVertexID id, final Configuration jobConfiguration,
- final RuntimeEnvironment environment, final Set<ChannelID> activeOutputChannels)
+ final RuntimeEnvironment environment)
throws InsufficientResourcesException, IOException {
if (id == null) {
@@ -730,10 +733,10 @@ public class TaskManager implements TaskOperationProtocol {
final Task runningTask = this.runningTasks.get(id);
boolean registerTask = true;
if (runningTask == null) {
- task = new RuntimeTask(id, environment, this);
+ task = new Task(id, environment, this);
} else {
- if (runningTask instanceof RuntimeTask) {
+ if (runningTask instanceof Task) {
// Task is already running
return null;
} else {
@@ -746,7 +749,7 @@ public class TaskManager implements TaskOperationProtocol {
if (registerTask) {
// Register the task with the byte buffered channel manager
- this.byteBufferedChannelManager.register(task, activeOutputChannels);
+ this.channelManager.register(task);
boolean enableProfiling = false;
if (this.profiler != null && jobConfiguration.getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true)) {
@@ -786,7 +789,7 @@ public class TaskManager implements TaskOperationProtocol {
this.fileCache.deleteTmpFile(e.getKey(), task.getJobID());
}
// Unregister task from the byte buffered channel manager
- this.byteBufferedChannelManager.unregister(id, task);
+ this.channelManager.unregister(id, task);
// Unregister task from profiling
task.unregisterProfiler(this.profiler);
@@ -892,7 +895,7 @@ public class TaskManager implements TaskOperationProtocol {
}
// Shut down the network channel manager
- this.byteBufferedChannelManager.shutdown();
+ this.channelManager.shutdown();
// Shut down the memory manager
if (this.ioManager != null) {
@@ -930,8 +933,9 @@ public class TaskManager implements TaskOperationProtocol {
}
@Override
- public void logBufferUtilization() {
- this.byteBufferedChannelManager.logBufferUtilization();
+ public void logBufferUtilization() throws IOException {
+
+ this.channelManager.logBufferUtilization();
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferAvailabilityListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferAvailabilityListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferAvailabilityListener.java
deleted file mode 100644
index 49e774b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferAvailabilityListener.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/***********************************************************************************************************************
- * 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.bufferprovider;
-
-/**
- * This interface must be implemented to receive a notification from a {@link BufferProvider} when an empty
- * {@link Buffer} has
- * become available again.
- *
- */
-public interface BufferAvailabilityListener {
-
- /**
- * Indicates that at least one {@link Buffer} has become available again.
- */
- void bufferAvailable();
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProvider.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProvider.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProvider.java
deleted file mode 100644
index 1b47b57..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProvider.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/***********************************************************************************************************************
- * 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.bufferprovider;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.io.channels.Buffer;
-
-public interface BufferProvider {
-
- /**
- * Requests an empty buffer with a minimum size of <code>minimumSizeOfBuffer</code>. The method returns
- * immediately, even if the request could not be fulfilled. Note that <code>minimumSizeOfBuffer</code> must not
- * exceed the value returned by the method <code>getMaximumBufferSize()</code>.
- *
- * @param minimumSizeOfBuffer
- * the minimum size of the requested read buffer in bytes
- * @return the buffer with at least the requested size or <code>null</code> if no such buffer is currently available
- * @throws IOException
- * thrown if an I/O error occurs while allocating the buffer
- */
- Buffer requestEmptyBuffer(int minimumSizeOfBuffer) throws IOException;
-
- /**
- * Requests an empty buffer with a minimum size of <code>minimumSizeOfBuffer</code>. The method blocks
- * until the request can be fulfilled. Note that <code>minimumSizeOfBuffer</code> must not
- * exceed the value returned by the method <code>getMaximumBufferSize()</code>.
- *
- * @param minimumSizeOfBuffer
- * the minimum size of the requested read buffer in bytes
- * @return the buffer with at least the requested size
- * @throws IOException
- * thrown if an I/O error occurs while allocating the buffer
- * @throws InterruptedException
- * thrown if the thread waiting for the buffer is interrupted
- */
- Buffer requestEmptyBufferBlocking(int minimumSizeOfBuffer) throws IOException,
- InterruptedException;
-
- /**
- * Returns the maximum buffer size in bytes available at this buffer provider.
- *
- * @return the maximum buffer size in bytes available at this buffer provider
- */
- int getMaximumBufferSize();
-
- /**
- * Returns if this buffer provider is shared between different entities (for examples tasks).
- *
- * @return <code>true</code> if this buffer provider is shared, <code>false</code> otherwise
- */
- boolean isShared();
-
- /**
- * Reports an asynchronous event. Calling this method interrupts each blocking method of the buffer provider and
- * allows the blocked thread to respond to the event.
- */
- void reportAsynchronousEvent();
-
- /**
- * Registers the given {@link BufferAvailabilityListener} with an empty buffer pool to receive a notification when
- * at least one buffer has become available again. After the notification, the listener is automatically
- * unregistered again.
- *
- * @param bufferAvailabilityListener
- * the listener to be registered
- * @return <code>true</code> if the registration has been successful or <code>false</code> if the registration
- * failed because the buffer pool has not been empty or has already been destroyed
- */
- boolean registerBufferAvailabilityListener(BufferAvailabilityListener bufferAvailabilityListener);
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProviderBroker.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProviderBroker.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProviderBroker.java
deleted file mode 100644
index 6474386..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/BufferProviderBroker.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/***********************************************************************************************************************
- * 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.bufferprovider;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-public interface BufferProviderBroker {
-
- BufferProvider getBufferProvider(JobID jobID, ChannelID sourceChannelID) throws IOException, InterruptedException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/GlobalBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/GlobalBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/GlobalBufferPool.java
deleted file mode 100644
index 0fc25eb..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/GlobalBufferPool.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/***********************************************************************************************************************
- * 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.bufferprovider;
-
-import java.util.Queue;
-import java.util.concurrent.ArrayBlockingQueue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.configuration.ConfigConstants;
-import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.core.memory.MemorySegment;
-
-public final class GlobalBufferPool {
-
- private final static Log LOG = LogFactory.getLog(GlobalBufferPool.class);
-
- /**
- * The singleton instance of the global buffer pool.
- */
- private static GlobalBufferPool instance = null;
-
- /**
- * The number of buffers created at startup.
- */
- private final int numberOfBuffers;
-
- /**
- * The size of read/write buffers in bytes.
- */
- private final int bufferSizeInBytes;
-
- private final Queue<MemorySegment> buffers;
-
- /**
- * Returns the singleton instance of the global buffer pool. If the instance does not already exist, it is also
- * created by calling this method.
- *
- * @return the singleton instance of the global buffer pool
- */
- public static synchronized GlobalBufferPool getInstance() {
-
- if (instance == null) {
- instance = new GlobalBufferPool();
- }
-
- return instance;
- }
-
- /**
- * Constructs the global buffer pool.
- */
- private GlobalBufferPool() {
-
- this.numberOfBuffers = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
- ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
- this.bufferSizeInBytes = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
- ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_BUFFER_SIZE);
-
- this.buffers = new ArrayBlockingQueue<MemorySegment>(this.numberOfBuffers);
-
- // Initialize buffers
- for (int i = 0; i < this.numberOfBuffers; i++) {
- // allocate byteBuffer
- final byte[] segMemory = new byte[this.bufferSizeInBytes];
- final MemorySegment readBuffer = new MemorySegment(segMemory);
- this.buffers.add(readBuffer);
- }
-
- LOG.info("Initialized global buffer pool with " + this.numberOfBuffers + " buffers with a size "
- + this.bufferSizeInBytes + " bytes each");
- }
-
- /**
- * Returns the maximum size of a buffer available at this pool in bytes.
- *
- * @return the maximum size of a buffer available at this pool in bytes
- */
- public int getMaximumBufferSize() {
-
- return this.bufferSizeInBytes;
- }
-
- /**
- * Locks a buffer from the global buffer pool and returns it to the caller of this method.
- *
- * @return the locked buffer from the pool or <code>null</code> if currently no global buffer is available
- */
- public MemorySegment lockGlobalBuffer() {
-
- return this.buffers.poll();
- }
-
- /**
- * Releases a lock on a previously locked buffer and returns the buffer to the global pool.
- *
- * @param releasedBuffer
- * the previously locked buffer to be released
- */
- public void releaseGlobalBuffer(final MemorySegment releasedBuffer) {
- this.buffers.add(releasedBuffer);
- }
-
- /**
- * Returns the total number of buffers managed by this pool.
- *
- * @return the total number of buffers managed by this pool
- */
- public int getTotalNumberOfBuffers() {
-
- return this.numberOfBuffers;
- }
-
- /**
- * Returns the number of buffers which are currently available at this pool.
- *
- * @return the number of buffers which are currently available at this pool
- */
- public int getCurrentNumberOfBuffers() {
-
- return this.buffers.size();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java
deleted file mode 100644
index f296003..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPool.java
+++ /dev/null
@@ -1,287 +0,0 @@
-/***********************************************************************************************************************
- * 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.bufferprovider;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.Queue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.BufferFactory;
-import eu.stratosphere.nephele.io.channels.MemoryBufferPoolConnector;
-
-public final class LocalBufferPool implements BufferProvider {
-
- private static final class LocalBufferPoolConnector implements MemoryBufferPoolConnector {
-
- private final LocalBufferPool localBufferPool;
-
- private LocalBufferPoolConnector(final LocalBufferPool localBufferPool) {
- this.localBufferPool = localBufferPool;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public void recycle(final MemorySegment byteBuffer) {
-
- this.localBufferPool.recycleBuffer(byteBuffer);
- }
-
- }
-
- private final static Log LOG = LogFactory.getLog(LocalBufferPool.class);
-
- private final GlobalBufferPool globalBufferPool;
-
- private final int maximumBufferSize;
-
- private int designatedNumberOfBuffers;
-
- private int requestedNumberOfBuffers = 0;
-
- private final boolean isShared;
-
- private boolean asynchronousEventOccurred = false;
-
- private boolean isDestroyed = false;
-
- private final Queue<MemorySegment> buffers = new ArrayDeque<MemorySegment>();
-
- private final LocalBufferPoolConnector bufferPoolConnector;
-
- private final Queue<BufferAvailabilityListener> bufferAvailabilityListenerQueue = new ArrayDeque<BufferAvailabilityListener>();
-
- public LocalBufferPool(final int designatedNumberOfBuffers, final boolean isShared) {
-
- this.globalBufferPool = GlobalBufferPool.getInstance();
- this.maximumBufferSize = this.globalBufferPool.getMaximumBufferSize();
- this.designatedNumberOfBuffers = designatedNumberOfBuffers;
- this.isShared = isShared;
- this.bufferPoolConnector = new LocalBufferPoolConnector(this);
- }
-
-
- @Override
- public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException {
-
- try {
- return requestBufferInternal(minimumSizeOfBuffer, false);
- } catch (InterruptedException e) {
- LOG.error("Caught unexpected InterruptedException");
- }
-
- return null;
- }
-
-
- @Override
- public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws IOException, InterruptedException {
-
- return requestBufferInternal(minimumSizeOfBuffer, true);
- }
-
- private Buffer requestBufferInternal(final int minimumSizeOfBuffer, final boolean block) throws IOException,
- InterruptedException {
-
- if (minimumSizeOfBuffer > this.maximumBufferSize) {
- throw new IllegalArgumentException("Buffer of " + minimumSizeOfBuffer
- + " bytes is requested, but maximum buffer size is " + this.maximumBufferSize);
- }
-
- while (true) {
-
- boolean async = false;
-
- synchronized (this.buffers) {
-
- // Make sure we return excess buffers immediately
- while (this.requestedNumberOfBuffers > this.designatedNumberOfBuffers) {
-
- final MemorySegment seg = this.buffers.poll();
- if (seg == null) {
- break;
- }
-
- this.globalBufferPool.releaseGlobalBuffer(seg);
- this.requestedNumberOfBuffers--;
- }
-
- while (this.buffers.isEmpty()) {
-
- // Check if the number of cached buffers matches the number of designated buffers
- if (this.requestedNumberOfBuffers < this.designatedNumberOfBuffers) {
-
- final MemorySegment memSeg = this.globalBufferPool.lockGlobalBuffer();
- if (memSeg != null) {
- this.buffers.add(memSeg);
- this.requestedNumberOfBuffers++;
- continue;
- }
- }
-
- if (this.asynchronousEventOccurred && block) {
- this.asynchronousEventOccurred = false;
- async = true;
- break;
- }
-
- if (block) {
- this.buffers.wait(100);
- } else {
- return null;
- }
- }
-
- if (!async) {
- final MemorySegment memSeg = this.buffers.poll();
- return BufferFactory.createFromMemory(minimumSizeOfBuffer, memSeg, this.bufferPoolConnector);
- }
- }
- }
- }
-
-
- @Override
- public int getMaximumBufferSize() {
-
- return this.maximumBufferSize;
- }
-
- /**
- * Sets the designated number of buffers for this local buffer cache.
- *
- * @param designatedNumberOfBuffers
- * the designated number of buffers for this local buffer cache
- */
- public void setDesignatedNumberOfBuffers(final int designatedNumberOfBuffers) {
-
- synchronized (this.buffers) {
-
- this.designatedNumberOfBuffers = designatedNumberOfBuffers;
-
- // Make sure we return excess buffers immediately
- while (this.requestedNumberOfBuffers > this.designatedNumberOfBuffers) {
-
- if (this.buffers.isEmpty()) {
- break;
- }
-
- this.globalBufferPool.releaseGlobalBuffer(this.buffers.poll());
- this.requestedNumberOfBuffers--;
- }
-
- this.buffers.notify();
- }
- }
-
- public void destroy() {
-
- synchronized (this.buffers) {
-
- if (this.isDestroyed) {
- LOG.error("destroy is called on LocalBufferPool multiple times");
- return;
- }
-
- this.isDestroyed = true;
-
- while (!this.buffers.isEmpty()) {
- this.globalBufferPool.releaseGlobalBuffer(this.buffers.poll());
- }
-
- this.requestedNumberOfBuffers = 0;
- }
- }
-
-
- @Override
- public boolean isShared() {
-
- return this.isShared;
- }
-
- public int getNumberOfAvailableBuffers() {
-
- synchronized (this.buffers) {
- return this.buffers.size();
- }
- }
-
- public int getDesignatedNumberOfBuffers() {
-
- synchronized (this.buffers) {
- return this.designatedNumberOfBuffers;
- }
- }
-
- public int getRequestedNumberOfBuffers() {
-
- synchronized (this.buffers) {
- return this.requestedNumberOfBuffers;
- }
- }
-
- private void recycleBuffer(final MemorySegment memSeg) {
-
- synchronized (this.buffers) {
-
- if (this.isDestroyed) {
- this.globalBufferPool.releaseGlobalBuffer(memSeg);
- this.requestedNumberOfBuffers--;
- } else {
- this.buffers.add(memSeg);
- this.buffers.notify();
- }
-
- while (!this.bufferAvailabilityListenerQueue.isEmpty()) {
- this.bufferAvailabilityListenerQueue.poll().bufferAvailable();
- }
- }
- }
-
-
- @Override
- public void reportAsynchronousEvent() {
-
- synchronized (this.buffers) {
- this.asynchronousEventOccurred = true;
- this.buffers.notify();
- }
- }
-
-
- @Override
- public boolean registerBufferAvailabilityListener(final BufferAvailabilityListener bufferAvailabilityListener) {
-
- synchronized (this.buffers) {
- if (!this.buffers.isEmpty()) {
- return false;
- }
-
- if (this.isDestroyed) {
- return false;
- }
-
- this.bufferAvailabilityListenerQueue.add(bufferAvailabilityListener);
- }
-
- return true;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPoolOwner.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPoolOwner.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPoolOwner.java
deleted file mode 100644
index c56a73c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bufferprovider/LocalBufferPoolOwner.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/***********************************************************************************************************************
- * 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.bufferprovider;
-
-/**
- * A local buffer pool owner is an object which initially retrieves its buffers from the {@link GlobalBufferPool} and
- * manages its fraction of the overall buffer pool locally by means of a {@link LocalBufferPool}.
- *
- */
-public interface LocalBufferPoolOwner {
-
- /**
- * Returns the number of byte-buffered channels that will retrieve their buffers from the local buffer pool.
- *
- * @return the number of byte-buffered channels that will retrieve their buffers from the local buffer pool
- */
- int getNumberOfChannels();
-
- /**
- * Sets the designated number of buffers the local buffer pool owner is allowed to fetch from the global buffer pool
- * and manage locally by means of the {@link LocalBufferPool}.
- *
- * @param numberOfBuffers
- * the numberOfBuffers the local buffer pool owner is allowed to fetch from the global buffer pool
- */
- void setDesignatedNumberOfBuffers(int numberOfBuffers);
-
- /**
- * Clears the local buffer pool and returns all buffers to the global buffer pool.
- */
- void clearLocalBufferPool();
-
- /**
- * Logs the current status of the local buffer pool. This method is intended mainly for debugging purposes.
- */
- void logBufferUtilization();
-
- /**
- * Reports an asynchronous event. Calling this method interrupts each blocking method of the buffer pool owner and
- * allows the blocked thread to respond to the event.
- */
- void reportAsynchronousEvent();
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java
deleted file mode 100644
index 066c268..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelContext.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import java.util.Iterator;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-public abstract class AbstractOutputChannelContext implements OutputChannelContext {
-
- /**
- * The forwarding chain used by this output channel context.
- */
- private final OutputChannelForwardingChain forwardingChain;
-
- public AbstractOutputChannelContext(final OutputChannelForwardingChain forwardingChain) {
-
- this.forwardingChain = forwardingChain;
- }
-
-
- @Override
- public void queueTransferEnvelope(final TransferEnvelope transferEnvelope) {
-
- if (transferEnvelope.getBuffer() != null) {
- throw new IllegalStateException("Transfer envelope for output channel has buffer attached");
- }
-
- final Iterator<AbstractEvent> it = transferEnvelope.getEventList().iterator();
- while (it.hasNext()) {
-
- final AbstractEvent event = it.next();
- if (event instanceof AbstractTaskEvent) {
- processEventAsynchronously(event);
- } else {
- processEventSynchronously(event);
- }
- }
- }
-
- /**
- * Processes an event received from the framework in a synchronous fashion, i.e. the event processing is done by the
- * thread the event is destined for (usually the task thread).
- *
- * @param event
- * the event to be processed
- */
- protected void processEventSynchronously(final AbstractEvent event) {
-
- this.forwardingChain.offerEvent(event);
- }
-
- /**
- * Processes an event received from the framework in an asynchronous fashion, i.e. the event processing is done by
- * the thread which delivers the event.
- *
- * @param event
- * the event to be processed
- */
- protected void processEventAsynchronously(final AbstractEvent event) {
-
- // The default implementation does nothing
- }
-
-
- @Override
- public void destroy() {
-
- this.forwardingChain.destroy();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelForwarder.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelForwarder.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelForwarder.java
deleted file mode 100644
index 4a59cdc..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/AbstractOutputChannelForwarder.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-/**
- * An output channel forwarder is a component which processes a {@link TransferEnvelope} after it has been produced by
- * an {@link AbstractByteBufferedOutputChannel}. The component can decide based on the transfer envelope whether to
- * forward the envelope, discard it, or to store it.
- *
- */
-public abstract class AbstractOutputChannelForwarder {
-
- private final AbstractOutputChannelForwarder next;
-
- private volatile AbstractOutputChannelForwarder prev = null;
-
- protected AbstractOutputChannelForwarder(final AbstractOutputChannelForwarder next) {
- this.next = next;
- if (this.next != null) {
- this.next.prev = this;
- }
- }
-
- /**
- * Called by the framework to push a produced transfer envelope towards its receiver. This method will always be
- * called by the task thread itself.
- *
- * @param transferEnvelope
- * the transfer envelope to be processed
- * @throws IOException
- * thrown if an I/O error occurs while processing the transfer envelope
- * @throws InterruptedException
- * thrown if the task thread was interrupted while processing the transfer envelope
- */
- public void push(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException {
-
- if (this.next != null) {
- this.next.push(transferEnvelope);
- }
- }
-
- public TransferEnvelope pull() {
-
- if (this.prev != null) {
- return this.prev.pull();
- }
-
- return null;
- }
-
- public boolean hasDataLeft() throws IOException, InterruptedException {
-
- if (this.next != null) {
- this.next.hasDataLeft();
- }
-
- return false;
- }
-
- public void processEvent(final AbstractEvent event) {
-
- if (this.next != null) {
- this.next.processEvent(event);
- }
- }
-
- public void destroy() {
-
- if (this.next != null) {
- this.next.destroy();
- }
- }
-
- protected final void recycleTransferEnvelope(final TransferEnvelope transferEnvelope) {
-
- final Buffer buffer = transferEnvelope.getBuffer();
- if (buffer != null) {
- buffer.recycleBuffer();
-
- }
- }
-
- protected final AbstractOutputChannelForwarder getNext() {
-
- return this.next;
- }
-
- protected final AbstractOutputChannelForwarder getPrev() {
-
- return this.prev;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java
deleted file mode 100644
index 3bf351d..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ByteBufferedChannelManager.java
+++ /dev/null
@@ -1,816 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.nephele.execution.Environment;
-import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
-import eu.stratosphere.nephele.instance.InstanceConnectionInfo;
-import eu.stratosphere.nephele.io.AbstractID;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.protocols.ChannelLookupProtocol;
-import eu.stratosphere.nephele.taskmanager.Task;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProviderBroker;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.GlobalBufferPool;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeReceiverList;
-
-public final class ByteBufferedChannelManager implements TransferEnvelopeDispatcher, BufferProviderBroker {
-
- /**
- * The log object used to report problems and errors.
- */
- private static final Log LOG = LogFactory.getLog(ByteBufferedChannelManager.class);
-
- private static final boolean DEFAULT_ALLOW_SENDER_SIDE_SPILLING = false;
-
- private static final boolean DEFAULT_MERGE_SPILLED_BUFFERS = true;
-
- // TODO: Make this configurable
- private static final int NUMBER_OF_CHANNELS_FOR_MULTICAST = 10;
-
- private final Map<ChannelID, ChannelContext> registeredChannels = new ConcurrentHashMap<ChannelID, ChannelContext>();
-
- private final Map<AbstractID, LocalBufferPoolOwner> localBufferPoolOwner = new ConcurrentHashMap<AbstractID, LocalBufferPoolOwner>();
-
- private final NetworkConnectionManager networkConnectionManager;
-
- private final ChannelLookupProtocol channelLookupService;
-
- private final InstanceConnectionInfo localConnectionInfo;
-
- private final LocalBufferPool transitBufferPool;
-
- private final boolean allowSenderSideSpilling;
-
- private final boolean mergeSpilledBuffers;
-
- private final boolean multicastEnabled = true;
-
- /**
- * This map caches transfer envelope receiver lists.
- */
- private final Map<ChannelID, TransferEnvelopeReceiverList> receiverCache = new ConcurrentHashMap<ChannelID, TransferEnvelopeReceiverList>();
-
- public ByteBufferedChannelManager(final ChannelLookupProtocol channelLookupService,
- final InstanceConnectionInfo localInstanceConnectionInfo) throws IOException {
-
- this.channelLookupService = channelLookupService;
-
- this.localConnectionInfo = localInstanceConnectionInfo;
-
- // Initialize the global buffer pool
- GlobalBufferPool.getInstance();
-
- // Initialize the transit buffer pool
- this.transitBufferPool = new LocalBufferPool(128, true);
-
- this.networkConnectionManager = new NetworkConnectionManager(this,
- localInstanceConnectionInfo.getAddress(), localInstanceConnectionInfo.getDataPort());
-
- this.allowSenderSideSpilling = GlobalConfiguration.getBoolean("channel.network.allowSenderSideSpilling",
- DEFAULT_ALLOW_SENDER_SIDE_SPILLING);
-
- this.mergeSpilledBuffers = GlobalConfiguration.getBoolean("channel.network.mergeSpilledBuffers",
- DEFAULT_MERGE_SPILLED_BUFFERS);
-
- LOG.info("Initialized byte buffered channel manager with sender-side spilling "
- + (this.allowSenderSideSpilling ? "enabled" : "disabled")
- + (this.mergeSpilledBuffers ? " and spilled buffer merging enabled" : ""));
- }
-
- /**
- * Registers the given task with the byte buffered channel manager.
- *
- * @param task
- * the task to be registered
- * @param the
- * set of output channels which are initially active
- * @throws InsufficientResourcesException
- * thrown if the channel manager does not have enough memory buffers to safely run this task
- */
- public void register(final Task task, final Set<ChannelID> activeOutputChannels)
- throws InsufficientResourcesException {
-
- // Check if we can safely run this task with the given resources
- checkBufferAvailability(task);
-
- final Environment environment = task.getEnvironment();
-
- final TaskContext taskContext = task.createTaskContext(this,
- this.localBufferPoolOwner.remove(task.getVertexID()));
-
- final Set<GateID> outputGateIDs = environment.getOutputGateIDs();
- for (final Iterator<GateID> gateIt = outputGateIDs.iterator(); gateIt.hasNext();) {
-
- final GateID gateID = gateIt.next();
- final OutputGateContext outputGateContext = taskContext.createOutputGateContext(gateID);
- final Set<ChannelID> outputChannelIDs = environment.getOutputChannelIDsOfGate(gateID);
- for (final Iterator<ChannelID> channelIt = outputChannelIDs.iterator(); channelIt.hasNext();) {
-
- final ChannelID channelID = channelIt.next();
- final OutputChannelContext previousContext = (OutputChannelContext) this.registeredChannels
- .get(channelID);
-
- final boolean isActive = true;/* activeOutputChannels.contains(channelID); */
-
- final OutputChannelContext outputChannelContext = outputGateContext.createOutputChannelContext(
- channelID, previousContext, isActive, this.mergeSpilledBuffers);
-
- // Add routing entry to receiver cache to reduce latency
- if (outputChannelContext.getType() == ChannelType.INMEMORY) {
- addReceiverListHint(outputChannelContext.getChannelID(),
- outputChannelContext.getConnectedChannelID());
- }
-
- // Add routing entry to receiver cache to save lookup for data arriving at the output channel
- if (outputChannelContext.getType() == ChannelType.NETWORK) {
- addReceiverListHint(outputChannelContext.getConnectedChannelID(),
- outputChannelContext.getChannelID());
- }
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Registering byte buffered output channel " + outputChannelContext.getChannelID() + " ("
- + (isActive ? "active" : "inactive") + ")");
- }
-
- this.registeredChannels.put(outputChannelContext.getChannelID(), outputChannelContext);
- }
- }
-
- final Set<GateID> inputGateIDs = environment.getInputGateIDs();
- for (final Iterator<GateID> gateIt = inputGateIDs.iterator(); gateIt.hasNext();) {
-
- final GateID gateID = gateIt.next();
- final InputGateContext inputGateContext = taskContext.createInputGateContext(gateID);
- final Set<ChannelID> inputChannelIDs = environment.getInputChannelIDsOfGate(gateID);
- for (final Iterator<ChannelID> channelIt = inputChannelIDs.iterator(); channelIt.hasNext();) {
-
- final ChannelID channelID = channelIt.next();
- final InputChannelContext previousContext = (InputChannelContext) this.registeredChannels
- .get(channelID);
-
- final InputChannelContext inputChannelContext = inputGateContext.createInputChannelContext(
- channelID, previousContext);
-
- // Add routing entry to receiver cache to reduce latency
- if (inputChannelContext.getType() == ChannelType.INMEMORY) {
- addReceiverListHint(inputChannelContext.getChannelID(), inputChannelContext.getConnectedChannelID());
- }
-
- this.registeredChannels.put(inputChannelContext.getChannelID(), inputChannelContext);
- }
-
- // Add input gate context to set of local buffer pool owner
- final LocalBufferPoolOwner bufferPoolOwner = inputGateContext.getLocalBufferPoolOwner();
- if (bufferPoolOwner != null) {
- this.localBufferPoolOwner.put(inputGateContext.getGateID(), bufferPoolOwner);
- }
-
- }
-
- this.localBufferPoolOwner.put(task.getVertexID(), taskContext);
-
- redistributeGlobalBuffers();
- }
-
- /**
- * Unregisters the given task from the byte buffered channel manager.
- *
- * @param vertexID
- * the ID of the task to be unregistered
- * @param task
- * the task to be unregistered
- */
- public void unregister(final ExecutionVertexID vertexID, final Task task) {
-
- final Environment environment = task.getEnvironment();
-
- Iterator<ChannelID> channelIterator = environment.getOutputChannelIDs().iterator();
-
- while (channelIterator.hasNext()) {
-
- final ChannelID outputChannelID = channelIterator.next();
- final ChannelContext context = this.registeredChannels.remove(outputChannelID);
- if (context != null) {
- context.destroy();
- }
- this.receiverCache.remove(outputChannelID);
- }
-
- channelIterator = environment.getInputChannelIDs().iterator();
-
- while (channelIterator.hasNext()) {
-
- final ChannelID outputChannelID = channelIterator.next();
- final ChannelContext context = this.registeredChannels.remove(outputChannelID);
- if (context != null) {
- context.destroy();
- }
- this.receiverCache.remove(outputChannelID);
- }
-
- final Iterator<GateID> inputGateIterator = environment.getInputGateIDs().iterator();
-
- while (inputGateIterator.hasNext()) {
-
- final GateID inputGateID = inputGateIterator.next();
-
- final LocalBufferPoolOwner owner = this.localBufferPoolOwner.remove(inputGateID);
- if (owner != null) {
- owner.clearLocalBufferPool();
- }
- }
-
- final LocalBufferPoolOwner owner = this.localBufferPoolOwner.remove(vertexID);
- if (owner != null) {
- owner.clearLocalBufferPool();
- }
-
- redistributeGlobalBuffers();
- }
-
- /**
- * Shuts down the byte buffered channel manager and stops all its internal processes.
- */
- public void shutdown() {
-
- this.networkConnectionManager.shutDown();
- }
-
- public NetworkConnectionManager getNetworkConnectionManager() {
-
- return this.networkConnectionManager;
- }
-
- private void recycleBuffer(final TransferEnvelope envelope) {
-
- final Buffer buffer = envelope.getBuffer();
- if (buffer != null) {
- buffer.recycleBuffer();
- }
- }
-
- private void sendReceiverNotFoundEvent(final TransferEnvelope envelope, final ChannelID receiver)
- throws IOException, InterruptedException {
-
- if (ReceiverNotFoundEvent.isReceiverNotFoundEvent(envelope)) {
-
- LOG.info("Dropping request to send ReceiverNotFoundEvent as response to ReceiverNotFoundEvent");
- return;
- }
-
- final JobID jobID = envelope.getJobID();
-
- final TransferEnvelope transferEnvelope = ReceiverNotFoundEvent.createEnvelopeWithEvent(jobID, receiver,
- envelope.getSequenceNumber());
-
- final TransferEnvelopeReceiverList receiverList = getReceiverList(jobID, receiver);
- if (receiverList == null) {
- return;
- }
-
- processEnvelopeEnvelopeWithoutBuffer(transferEnvelope, receiverList);
- }
-
- private void processEnvelope(final TransferEnvelope transferEnvelope, final boolean freeSourceBuffer)
- throws IOException, InterruptedException {
-
- TransferEnvelopeReceiverList receiverList = null;
- try {
- receiverList = getReceiverList(transferEnvelope.getJobID(),
- transferEnvelope.getSource());
- } catch (InterruptedException e) {
- recycleBuffer(transferEnvelope);
- throw e;
- } catch (IOException e) {
- recycleBuffer(transferEnvelope);
- throw e;
- }
-
- if (receiverList == null) {
- recycleBuffer(transferEnvelope);
- return;
- }
-
- // This envelope is known to have either no buffer or an memory-based input buffer
- if (transferEnvelope.getBuffer() == null) {
- processEnvelopeEnvelopeWithoutBuffer(transferEnvelope, receiverList);
- } else {
- processEnvelopeWithBuffer(transferEnvelope, receiverList, freeSourceBuffer);
- }
- }
-
- private void processEnvelopeWithBuffer(final TransferEnvelope transferEnvelope,
- final TransferEnvelopeReceiverList receiverList, final boolean freeSourceBuffer)
- throws IOException, InterruptedException {
-
- // Handle the most common (unicast) case first
- if (!freeSourceBuffer) {
-
- final List<ChannelID> localReceivers = receiverList.getLocalReceivers();
- if (localReceivers.size() != 1) {
- LOG.error("Expected receiver list to have exactly one element");
- }
-
- final ChannelID localReceiver = localReceivers.get(0);
-
- final ChannelContext cc = this.registeredChannels.get(localReceiver);
- if (cc == null) {
-
- try {
- sendReceiverNotFoundEvent(transferEnvelope, localReceiver);
- } finally {
- recycleBuffer(transferEnvelope);
- }
- return;
- }
-
- if (!cc.isInputChannel()) {
- LOG.error("Local receiver " + localReceiver
- + " is not an input channel, but is supposed to accept a buffer");
- }
-
- cc.queueTransferEnvelope(transferEnvelope);
-
- return;
- }
-
- // This is the in-memory or multicast case
- final Buffer srcBuffer = transferEnvelope.getBuffer();
-
- try {
-
- if (receiverList.hasLocalReceivers()) {
-
- final List<ChannelID> localReceivers = receiverList.getLocalReceivers();
-
- for (final ChannelID localReceiver : localReceivers) {
-
- final ChannelContext cc = this.registeredChannels.get(localReceiver);
- if (cc == null) {
-
- sendReceiverNotFoundEvent(transferEnvelope, localReceiver);
- continue;
- }
-
- if (!cc.isInputChannel()) {
- LOG.error("Local receiver " + localReceiver
- + " is not an input channel, but is supposed to accept a buffer");
- continue;
- }
-
- final InputChannelContext inputChannelContext = (InputChannelContext) cc;
-
- Buffer destBuffer = null;
- try {
- destBuffer = inputChannelContext.requestEmptyBufferBlocking(srcBuffer.size());
- srcBuffer.copyToBuffer(destBuffer);
- } catch (IOException e) {
- if (destBuffer != null) {
- destBuffer.recycleBuffer();
- }
- throw e;
- }
- // TODO: See if we can save one duplicate step here
- final TransferEnvelope dup = transferEnvelope.duplicateWithoutBuffer();
- dup.setBuffer(destBuffer);
- inputChannelContext.queueTransferEnvelope(dup);
- }
- }
-
- if (receiverList.hasRemoteReceivers()) {
-
- final List<RemoteReceiver> remoteReceivers = receiverList.getRemoteReceivers();
-
- // Generate sender hint before sending the first envelope over the network
- if (transferEnvelope.getSequenceNumber() == 0) {
- generateSenderHint(transferEnvelope, remoteReceivers);
- }
-
- for (final RemoteReceiver remoteReceiver : remoteReceivers) {
- TransferEnvelope dup = transferEnvelope.duplicate();
- this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, dup);
- }
- }
- } finally {
- // Recycle the source buffer
- srcBuffer.recycleBuffer();
- }
- }
-
- private void processEnvelopeEnvelopeWithoutBuffer(final TransferEnvelope transferEnvelope,
- final TransferEnvelopeReceiverList receiverList) throws IOException, InterruptedException {
-
- // No need to copy anything
- final Iterator<ChannelID> localIt = receiverList.getLocalReceivers().iterator();
-
- while (localIt.hasNext()) {
-
- final ChannelID localReceiver = localIt.next();
-
- final ChannelContext channelContext = this.registeredChannels.get(localReceiver);
- if (channelContext == null) {
- sendReceiverNotFoundEvent(transferEnvelope, localReceiver);
- continue;
- }
- channelContext.queueTransferEnvelope(transferEnvelope);
- }
-
- if (!receiverList.hasRemoteReceivers()) {
- return;
- }
-
- // Generate sender hint before sending the first envelope over the network
- final List<RemoteReceiver> remoteReceivers = receiverList.getRemoteReceivers();
- if (transferEnvelope.getSequenceNumber() == 0) {
- generateSenderHint(transferEnvelope, remoteReceivers);
- }
-
- final Iterator<RemoteReceiver> remoteIt = remoteReceivers.iterator();
-
- while (remoteIt.hasNext()) {
-
- final RemoteReceiver remoteReceiver = remoteIt.next();
- this.networkConnectionManager.queueEnvelopeForTransfer(remoteReceiver, transferEnvelope);
- }
- }
-
- private void addReceiverListHint(final ChannelID source, final ChannelID localReceiver) {
-
- final TransferEnvelopeReceiverList receiverList = new TransferEnvelopeReceiverList(localReceiver);
-
- if (this.receiverCache.put(source, receiverList) != null) {
- LOG.warn("Receiver cache already contained entry for " + source);
- }
- }
-
- private void addReceiverListHint(final ChannelID source, final RemoteReceiver remoteReceiver) {
-
- final TransferEnvelopeReceiverList receiverList = new TransferEnvelopeReceiverList(remoteReceiver);
-
- if (this.receiverCache.put(source, receiverList) != null) {
- LOG.warn("Receiver cache already contained entry for " + source);
- }
- }
-
- private void generateSenderHint(final TransferEnvelope transferEnvelope, final List<RemoteReceiver> remoteReceivers) {
-
- final ChannelContext channelContext = this.registeredChannels.get(transferEnvelope.getSource());
- if (channelContext == null) {
- LOG.error("Cannot find channel context for channel ID " + transferEnvelope.getSource());
- return;
- }
-
- // Only generate sender hints for output channels
- if (channelContext.isInputChannel()) {
- return;
- }
-
- final ChannelID remoteSourceID = channelContext.getConnectedChannelID();
- final int connectionIndex = remoteReceivers.get(0).getConnectionIndex();
- final InetSocketAddress isa = new InetSocketAddress(this.localConnectionInfo.getAddress(),
- this.localConnectionInfo.getDataPort());
-
- final RemoteReceiver remoteReceiver = new RemoteReceiver(isa, connectionIndex);
- final TransferEnvelope senderHint = SenderHintEvent.createEnvelopeWithEvent(transferEnvelope, remoteSourceID,
- remoteReceiver);
-
- final Iterator<RemoteReceiver> remoteIt = remoteReceivers.iterator();
-
- while (remoteIt.hasNext()) {
-
- final RemoteReceiver rr = remoteIt.next();
- this.networkConnectionManager.queueEnvelopeForTransfer(rr, senderHint);
- }
- }
-
- /**
- * Returns the list of receivers for transfer envelopes produced by the channel with the given source channel ID.
- *
- * @param jobID
- * the ID of the job the given channel ID belongs to
- * @param sourceChannelID
- * 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
- * @throws IOException
- * @throws InterruptedExcption
- */
- private TransferEnvelopeReceiverList getReceiverList(final JobID jobID, final ChannelID sourceChannelID)
- throws IOException, InterruptedException {
-
- TransferEnvelopeReceiverList receiverList = this.receiverCache.get(sourceChannelID);
-
- if (receiverList != null) {
- return receiverList;
- }
-
- while (true) {
-
- if (Thread.currentThread().isInterrupted()) {
- break;
- }
-
- ConnectionInfoLookupResponse lookupResponse;
- synchronized (this.channelLookupService) {
- lookupResponse = this.channelLookupService.lookupConnectionInfo(
- this.localConnectionInfo, jobID, sourceChannelID);
- }
-
- if (lookupResponse.isJobAborting()) {
- break;
- }
-
- if (lookupResponse.receiverNotFound()) {
- LOG.error("Cannot find task(s) waiting for data from source channel with ID " + sourceChannelID);
- break;
- }
-
- if (lookupResponse.receiverNotReady()) {
- Thread.sleep(500);
- continue;
- }
-
- if (lookupResponse.receiverReady()) {
- receiverList = new TransferEnvelopeReceiverList(lookupResponse);
- break;
- }
-
- }
-
- if (receiverList != null) {
-
- this.receiverCache.put(sourceChannelID, receiverList);
-
- if (LOG.isDebugEnabled()) {
-
- final StringBuilder sb = new StringBuilder();
- sb.append("Receiver list for source channel ID " + sourceChannelID + " at task manager "
- + this.localConnectionInfo + "\n");
-
- if (receiverList.hasLocalReceivers()) {
- sb.append("\tLocal receivers:\n");
- final Iterator<ChannelID> it = receiverList.getLocalReceivers().iterator();
- while (it.hasNext()) {
- sb.append("\t\t" + it.next() + "\n");
- }
- }
-
- if (receiverList.hasRemoteReceivers()) {
- sb.append("Remote receivers:\n");
- final Iterator<RemoteReceiver> it = receiverList.getRemoteReceivers().iterator();
- while (it.hasNext()) {
- sb.append("\t\t" + it.next() + "\n");
- }
- }
-
- LOG.debug(sb.toString());
- }
- }
-
- return receiverList;
- }
-
-
- @Override
- public void processEnvelopeFromOutputChannel(final TransferEnvelope transferEnvelope) throws IOException,
- InterruptedException {
-
- processEnvelope(transferEnvelope, true);
- }
-
-
- @Override
- public void processEnvelopeFromInputChannel(final TransferEnvelope transferEnvelope) throws IOException,
- InterruptedException {
-
- processEnvelope(transferEnvelope, false);
- }
-
-
- @Override
- public void processEnvelopeFromNetwork(final TransferEnvelope transferEnvelope, boolean freeSourceBuffer)
- throws IOException, InterruptedException {
-
- // Check if the envelope is the special envelope with the sender hint event
- if (SenderHintEvent.isSenderHintEvent(transferEnvelope)) {
-
- // Check if this is the final destination of the sender hint event before adding it
- final SenderHintEvent seh = (SenderHintEvent) transferEnvelope.getEventList().get(0);
- if (this.registeredChannels.get(seh.getSource()) != null) {
-
- addReceiverListHint(seh.getSource(), seh.getRemoteReceiver());
- return;
- }
- }
-
- processEnvelope(transferEnvelope, freeSourceBuffer);
- }
-
- /**
- * Triggers the byte buffer channel manager write the current utilization of its read and write buffers to the logs.
- * This method is primarily for debugging purposes.
- */
- public void logBufferUtilization() {
-
- System.out.println("Buffer utilization at " + System.currentTimeMillis());
-
- System.out.println("\tUnused global buffers: " + GlobalBufferPool.getInstance().getCurrentNumberOfBuffers());
-
- System.out.println("\tLocal buffer pool status:");
-
- final Iterator<LocalBufferPoolOwner> it = this.localBufferPoolOwner.values().iterator();
- while (it.hasNext()) {
- it.next().logBufferUtilization();
- }
-
- this.networkConnectionManager.logBufferUtilization();
-
- System.out.println("\tIncoming connections:");
-
- final Iterator<Map.Entry<ChannelID, ChannelContext>> it2 = this.registeredChannels.entrySet()
- .iterator();
-
- while (it2.hasNext()) {
-
- final Map.Entry<ChannelID, ChannelContext> entry = it2.next();
- final ChannelContext context = entry.getValue();
- if (context.isInputChannel()) {
-
- final InputChannelContext inputChannelContext = (InputChannelContext) context;
- inputChannelContext.logQueuedEnvelopes();
- }
- }
- }
-
-
- @Override
- public BufferProvider getBufferProvider(final JobID jobID, final ChannelID sourceChannelID) throws IOException,
- InterruptedException {
-
- final TransferEnvelopeReceiverList receiverList = getReceiverList(jobID, sourceChannelID);
-
- // Receiver could not be determined, use transit buffer pool to read data from channel
- if (receiverList == null) {
- return this.transitBufferPool;
- }
-
- if (receiverList.hasLocalReceivers() && !receiverList.hasRemoteReceivers()) {
-
- final List<ChannelID> localReceivers = receiverList.getLocalReceivers();
- if (localReceivers.size() == 1) {
- // Unicast case, get final buffer provider
-
- final ChannelID localReceiver = localReceivers.get(0);
- final ChannelContext cc = this.registeredChannels.get(localReceiver);
- if (cc == null) {
-
- // Use the transit buffer for this purpose, data will be discarded in most cases anyway.
- return this.transitBufferPool;
- }
-
- if (!cc.isInputChannel()) {
- throw new IOException("Channel context for local receiver " + localReceiver
- + " is not an input channel context");
- }
-
- final InputChannelContext icc = (InputChannelContext) cc;
-
- return icc;
- }
- }
-
- return this.transitBufferPool;
- }
-
- /**
- * Checks if the byte buffered channel manager has enough resources available to safely execute the given task.
- *
- * @param task
- * the task to be executed
- * @throws InsufficientResourcesException
- * thrown if the byte buffered manager currently does not have enough resources available to execute the
- * task
- */
- private void checkBufferAvailability(final Task task) throws InsufficientResourcesException {
-
- final int totalNumberOfBuffers = GlobalBufferPool.getInstance().getTotalNumberOfBuffers();
- int numberOfAlreadyRegisteredChannels = this.registeredChannels.size();
- if (this.multicastEnabled) {
- numberOfAlreadyRegisteredChannels += NUMBER_OF_CHANNELS_FOR_MULTICAST;
- }
-
- final Environment env = task.getEnvironment();
-
- final int numberOfNewChannels = env.getNumberOfOutputChannels() + env.getNumberOfInputChannels();
- final int totalNumberOfChannels = numberOfAlreadyRegisteredChannels + numberOfNewChannels;
-
- final double buffersPerChannel = (double) totalNumberOfBuffers
- / (double) totalNumberOfChannels;
-
- if (buffersPerChannel < 1.0) {
-
- // Construct error message
- final StringBuilder sb = new StringBuilder(this.localConnectionInfo.getHostName());
- sb.append(" has not enough buffers available to safely execute ");
- sb.append(env.getTaskName());
- sb.append(" (");
- sb.append(totalNumberOfChannels - totalNumberOfBuffers);
- sb.append(" buffers are currently missing)");
-
- throw new InsufficientResourcesException(sb.toString());
- }
- }
-
- /**
- * Redistributes the global buffers among the registered tasks.
- */
- private void redistributeGlobalBuffers() {
-
- final int totalNumberOfBuffers = GlobalBufferPool.getInstance().getTotalNumberOfBuffers();
- int totalNumberOfChannels = this.registeredChannels.size();
- if (this.multicastEnabled) {
- totalNumberOfChannels += NUMBER_OF_CHANNELS_FOR_MULTICAST;
- }
- final double buffersPerChannel = (double) totalNumberOfBuffers / (double) totalNumberOfChannels;
- if (buffersPerChannel < 1.0) {
- LOG.warn("System is low on memory buffers. This may result in reduced performance.");
- }
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Total number of buffers is " + totalNumberOfBuffers);
- LOG.debug("Total number of channels is " + totalNumberOfChannels);
- }
-
- if (this.localBufferPoolOwner.isEmpty()) {
- return;
- }
-
- final Iterator<LocalBufferPoolOwner> it = this.localBufferPoolOwner.values().iterator();
- while (it.hasNext()) {
- final LocalBufferPoolOwner lbpo = it.next();
- lbpo.setDesignatedNumberOfBuffers((int) Math.ceil(buffersPerChannel * lbpo.getNumberOfChannels()));
- }
-
- if (this.multicastEnabled) {
- this.transitBufferPool.setDesignatedNumberOfBuffers((int) Math.ceil(buffersPerChannel
- * NUMBER_OF_CHANNELS_FOR_MULTICAST));
- }
- }
-
- /**
- * Invalidates the entries identified by the given channel IDs from the receiver lookup cache.
- *
- * @param channelIDs
- * the channel IDs identifying the cache entries to invalidate
- */
- public void invalidateLookupCacheEntries(final Set<ChannelID> channelIDs) {
-
- final Iterator<ChannelID> it = channelIDs.iterator();
- while (it.hasNext()) {
-
- this.receiverCache.remove(it.next());
- }
- }
-
- public void reportAsynchronousEvent(final ExecutionVertexID vertexID) {
-
- final LocalBufferPoolOwner lbpo = this.localBufferPoolOwner.get(vertexID);
- if (lbpo == null) {
- System.out.println("Cannot find local buffer pool owner for " + vertexID);
- return;
- }
-
- lbpo.reportAsynchronousEvent();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/CanceledChannelSet.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/CanceledChannelSet.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/CanceledChannelSet.java
deleted file mode 100644
index f1efdc7..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/CanceledChannelSet.java
+++ /dev/null
@@ -1,211 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-
-/**
- * This channel set stores the ID's of all channels whose tasks have been canceled recently. The set is cleaned up by
- * periodically calling the method <code>cleanup</code>.
- * <p>
- * This class is thread-safe.
- *
- */
-public final class CanceledChannelSet implements Set<ChannelID> {
-
- /**
- * The period of time the entries must at least remain in the map.
- */
- private final static long CLEANUP_INTERVAL = 30000; // 30 sec.
-
- /**
- * The map which stores the ID's of the channels whose tasks have been canceled.
- */
- private final Map<ChannelID, Long> canceledChannels = new HashMap<ChannelID, Long>();
-
-
- @Override
- public boolean add(ChannelID arg0) {
-
- final long now = System.currentTimeMillis();
-
- synchronized (this.canceledChannels) {
- if (this.canceledChannels.put(arg0, Long.valueOf(now)) == null) {
- return true;
- }
- }
-
- return false;
- }
-
-
- @Override
- public boolean addAll(Collection<? extends ChannelID> arg0) {
-
- final Long now = Long.valueOf(System.currentTimeMillis());
- final Iterator<? extends ChannelID> it = arg0.iterator();
- boolean retVal = false;
-
- synchronized (this.canceledChannels) {
-
- while (it.hasNext()) {
-
- if (this.canceledChannels.put(it.next(), now) == null) {
- retVal = true;
- }
- }
- }
-
- return retVal;
- }
-
-
- @Override
- public void clear() {
-
- synchronized (this.canceledChannels) {
- this.canceledChannels.clear();
- }
-
- }
-
-
- @Override
- public boolean contains(Object arg0) {
-
- synchronized (this.canceledChannels) {
- return this.canceledChannels.containsKey(arg0);
- }
- }
-
-
- @Override
- public boolean containsAll(Collection<?> arg0) {
-
- synchronized (this.canceledChannels) {
- return this.canceledChannels.keySet().containsAll(arg0);
- }
- }
-
-
- @Override
- public boolean isEmpty() {
-
- synchronized (this.canceledChannels) {
- return this.canceledChannels.isEmpty();
- }
- }
-
-
- @Override
- public Iterator<ChannelID> iterator() {
-
- synchronized (this.canceledChannels) {
- return this.canceledChannels.keySet().iterator();
- }
- }
-
-
- @Override
- public boolean remove(Object arg0) {
-
- synchronized (this.canceledChannels) {
- if (this.canceledChannels.remove(arg0) == null) {
- return false;
- }
- }
-
- return true;
- }
-
-
- @Override
- public boolean removeAll(Collection<?> arg0) {
-
- final Iterator<?> it = arg0.iterator();
- boolean retVal = false;
-
- synchronized (this.canceledChannels) {
-
- while (it.hasNext()) {
- if (this.canceledChannels.remove(it.next()) != null) {
- retVal = true;
- }
- }
- }
-
- return retVal;
- }
-
-
- @Override
- public boolean retainAll(Collection<?> arg0) {
-
- throw new RuntimeException("Method not implemented");
- }
-
-
- @Override
- public int size() {
-
- synchronized (this.canceledChannels) {
- return this.canceledChannels.size();
- }
- }
-
-
- @Override
- public Object[] toArray() {
-
- synchronized (this.canceledChannels) {
- return this.canceledChannels.keySet().toArray();
- }
- }
-
-
- @Override
- public <T> T[] toArray(T[] arg0) {
-
- synchronized (this.canceledChannels) {
- return this.canceledChannels.keySet().toArray(arg0);
- }
- }
-
- /**
- * Removes all entries from the set which have been added longer than <code>CLEANUP_INTERVAL</code> milliseconds
- * ago.
- */
- public void cleanup() {
-
- final long now = System.currentTimeMillis();
-
- synchronized (this.canceledChannels) {
-
- final Iterator<Map.Entry<ChannelID, Long>> it = this.canceledChannels.entrySet().iterator();
- while (it.hasNext()) {
-
- final Map.Entry<ChannelID, Long> entry = it.next();
- if ((entry.getValue().longValue() + CLEANUP_INTERVAL) < now) {
- it.remove();
- }
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java
deleted file mode 100644
index b3a9200..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ChannelContext.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-public interface ChannelContext {
-
- boolean isInputChannel();
-
- JobID getJobID();
-
- ChannelID getChannelID();
-
- ChannelID getConnectedChannelID();
-
- ChannelType getType();
-
- void queueTransferEnvelope(TransferEnvelope transferEnvelope);
-
- void destroy();
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java
deleted file mode 100644
index 69daa0e..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ConnectionInfoLookupResponse.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/***********************************************************************************************************************
- *
- * 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.bytebuffered;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.List;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.util.EnumUtils;
-import eu.stratosphere.nephele.util.SerializableArrayList;
-
-public class ConnectionInfoLookupResponse implements IOReadableWritable {
-
- private enum ReturnCode {
- NOT_FOUND, FOUND_AND_RECEIVER_READY, FOUND_BUT_RECEIVER_NOT_READY, JOB_IS_ABORTING
- };
-
- // was request successful?
- private ReturnCode returnCode;
-
- /**
- * Contains next-hop instances, this instance must forward multicast transmissions to.
- */
- private final SerializableArrayList<RemoteReceiver> remoteTargets = new SerializableArrayList<RemoteReceiver>();
-
- /**
- * Contains local ChannelIDs, multicast packets must be forwarded to.
- */
- private final SerializableArrayList<ChannelID> localTargets = new SerializableArrayList<ChannelID>();
-
- public ConnectionInfoLookupResponse() {
- this.returnCode = ReturnCode.NOT_FOUND;
- }
-
- public void addRemoteTarget(final RemoteReceiver remote) {
- this.remoteTargets.add(remote);
- }
-
- public void addLocalTarget(ChannelID local) {
- this.localTargets.add(local);
- }
-
- private void setReturnCode(ReturnCode code) {
- this.returnCode = code;
- }
-
- public List<RemoteReceiver> getRemoteTargets() {
- return this.remoteTargets;
- }
-
- public List<ChannelID> getLocalTargets() {
- return this.localTargets;
- }
-
- @Override
- public void read(DataInput in) throws IOException {
-
- this.localTargets.read(in);
- this.remoteTargets.read(in);
-
- this.returnCode = EnumUtils.readEnum(in, ReturnCode.class);
- }
-
- @Override
- public void write(DataOutput out) throws IOException {
-
- this.localTargets.write(out);
- this.remoteTargets.write(out);
-
- EnumUtils.writeEnum(out, this.returnCode);
-
- }
-
- public boolean receiverNotFound() {
-
- return (this.returnCode == ReturnCode.NOT_FOUND);
- }
-
- public boolean receiverNotReady() {
-
- return (this.returnCode == ReturnCode.FOUND_BUT_RECEIVER_NOT_READY);
- }
-
- public boolean receiverReady() {
-
- return (this.returnCode == ReturnCode.FOUND_AND_RECEIVER_READY);
- }
-
- public boolean isJobAborting() {
-
- return (this.returnCode == ReturnCode.JOB_IS_ABORTING);
- }
-
- public static ConnectionInfoLookupResponse createReceiverFoundAndReady(final ChannelID targetChannelID) {
-
- final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse();
- response.setReturnCode(ReturnCode.FOUND_AND_RECEIVER_READY);
- response.addLocalTarget(targetChannelID);
-
- return response;
- }
-
- public static ConnectionInfoLookupResponse createReceiverFoundAndReady(final RemoteReceiver remoteReceiver) {
-
- final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse();
- response.setReturnCode(ReturnCode.FOUND_AND_RECEIVER_READY);
- response.addRemoteTarget(remoteReceiver);
-
- return response;
- }
-
- /**
- * Constructor used to generate a plain ConnectionInfoLookupResponse object to be filled with multicast targets.
- *
- * @return
- */
- public static ConnectionInfoLookupResponse createReceiverFoundAndReady() {
-
- final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse();
- response.setReturnCode(ReturnCode.FOUND_AND_RECEIVER_READY);
-
- return response;
- }
-
- public static ConnectionInfoLookupResponse createReceiverNotFound() {
- final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse();
- response.setReturnCode(ReturnCode.NOT_FOUND);
-
- return response;
- }
-
- public static ConnectionInfoLookupResponse createReceiverNotReady() {
- final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse();
- response.setReturnCode(ReturnCode.FOUND_BUT_RECEIVER_NOT_READY);
-
- return response;
- }
-
- public static ConnectionInfoLookupResponse createJobIsAborting() {
- final ConnectionInfoLookupResponse response = new ConnectionInfoLookupResponse();
- response.setReturnCode(ReturnCode.JOB_IS_ABORTING);
-
- return response;
- }
-
- @Override
- public String toString() {
- StringBuilder returnstring = new StringBuilder();
- returnstring.append("local targets (total: " + this.localTargets.size() + "):\n");
- for (ChannelID i : this.localTargets) {
- returnstring.append(i + "\n");
- }
- returnstring.append("remote targets: (total: " + this.remoteTargets.size() + "):\n");
- for (final RemoteReceiver rr : this.remoteTargets) {
- returnstring.append(rr + "\n");
- }
- return returnstring.toString();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.java
deleted file mode 100644
index 7b8ce36..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/GateContext.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import eu.stratosphere.nephele.io.GateID;
-
-public interface GateContext {
-
- GateID getGateID();
-}
[18/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnection.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnection.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnection.java
deleted file mode 100644
index d0fa683..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnection.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import java.io.IOException;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.SelectionKey;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.DefaultDeserializer;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.NoBufferAvailableException;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-import eu.stratosphere.util.StringUtils;
-
-/**
- * This class represents an incoming data connection through which data streams are read and transformed into
- * {@link TransferEnvelope} objects. The source of the data stream is a TCP connection.
- *
- */
-public class IncomingConnection {
-
- /**
- * The log object used to report debug information and possible errors.
- */
- private static final Log LOG = LogFactory.getLog(IncomingConnection.class);
-
- /**
- * The readable byte channel through which the input data is retrieved.
- */
- private final ReadableByteChannel readableByteChannel;
-
- /**
- * The {@link DefaultDeserializer} used to transform the read bytes into transfer envelopes which can be
- * passed on to the respective channels.
- */
- private final DefaultDeserializer deserializer;
-
- /**
- * The byte buffered channel manager which handles and dispatches the received transfer envelopes.
- */
- private final ByteBufferedChannelManager byteBufferedChannelManager;
-
- public IncomingConnection(ByteBufferedChannelManager byteBufferedChannelManager,
- ReadableByteChannel readableByteChannel) {
- this.byteBufferedChannelManager = byteBufferedChannelManager;
- this.deserializer = new DefaultDeserializer(byteBufferedChannelManager);
- this.readableByteChannel = readableByteChannel;
- }
-
- public void reportTransmissionProblem(SelectionKey key, IOException ioe) {
-
- LOG.error(StringUtils.stringifyException(ioe));
-
- try {
- this.readableByteChannel.close();
- } catch (IOException e) {
- LOG.debug("An error occurred while closing the byte channel");
- }
-
- // Cancel key
- if (key != null) {
- key.cancel();
- }
-
- // Recycle read buffer
- if (this.deserializer.getBuffer() != null) {
- this.deserializer.getBuffer().recycleBuffer();
- }
-
- this.deserializer.reset();
- }
-
- public void read() throws IOException, InterruptedException, NoBufferAvailableException {
-
- this.deserializer.read(this.readableByteChannel);
-
- final TransferEnvelope transferEnvelope = this.deserializer.getFullyDeserializedTransferEnvelope();
- if (transferEnvelope != null) {
-
- final BufferProvider bufferProvider = this.deserializer.getBufferProvider();
- if (bufferProvider == null) {
- this.byteBufferedChannelManager.processEnvelopeFromNetwork(transferEnvelope, false);
- } else {
- this.byteBufferedChannelManager.processEnvelopeFromNetwork(transferEnvelope, bufferProvider.isShared());
- }
- }
- }
-
- public boolean isCloseUnexpected() {
-
- return this.deserializer.hasUnfinishedData();
- }
-
- public ReadableByteChannel getReadableByteChannel() {
- return this.readableByteChannel;
- }
-
- public void closeConnection(SelectionKey key) {
-
- try {
- this.readableByteChannel.close();
- } catch (IOException ioe) {
- LOG.error("On IOException occured while closing the socket: + " + StringUtils.stringifyException(ioe));
- }
-
- // Cancel key
- if (key != null) {
- key.cancel();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnectionThread.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnectionThread.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnectionThread.java
deleted file mode 100644
index 05c3326..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/IncomingConnectionThread.java
+++ /dev/null
@@ -1,223 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-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;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferAvailabilityListener;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.NoBufferAvailableException;
-import eu.stratosphere.util.StringUtils;
-
-public class IncomingConnectionThread extends Thread {
-
- private static final Log LOG = LogFactory.getLog(IncomingConnectionThread.class);
-
- private final ByteBufferedChannelManager byteBufferedChannelManager;
-
- 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;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public void bufferAvailable() {
-
- synchronized (this.pendingReadEventSubscribeRequests) {
- this.pendingReadEventSubscribeRequests.add(this.key);
- }
- }
- }
-
- public IncomingConnectionThread(ByteBufferedChannelManager byteBufferedChannelManager,
- boolean isListeningThread, InetSocketAddress listeningAddress) throws IOException {
- super("Incoming Connection Thread");
-
- this.selector = Selector.open();
- this.byteBufferedChannelManager = byteBufferedChannelManager;
-
- 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() {
-
- 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));
- }
- }
-
- 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(this.byteBufferedChannelManager,
- clientSocket);
- 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);
- }
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java
deleted file mode 100644
index 739ec3d..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputChannelContext.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-
-public interface InputChannelContext extends ChannelContext, BufferProvider {
-
- void logQueuedEnvelopes();
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputGateContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputGateContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputGateContext.java
deleted file mode 100644
index 0b05262..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InputGateContext.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner;
-
-public interface InputGateContext extends GateContext {
-
- InputChannelContext createInputChannelContext(ChannelID channelID, InputChannelContext previousContext);
-
- LocalBufferPoolOwner getLocalBufferPoolOwner();
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InsufficientResourcesException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InsufficientResourcesException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InsufficientResourcesException.java
deleted file mode 100644
index 87543b8..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/InsufficientResourcesException.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-/**
- * This exception is thrown by the {@link ByteBufferedChannelManager} to indicate that a task cannot be accepted because
- * there are not enough resources available to safely execute it.
- *
- */
-public final class InsufficientResourcesException extends Exception {
-
- /**
- * The generated serial version UID.
- */
- private static final long serialVersionUID = -8977049569413215169L;
-
- /**
- * Constructs a new insufficient resources exception.
- *
- * @param msg
- * the message describing the exception
- */
- InsufficientResourcesException(final String msg) {
- super(msg);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/NetworkConnectionManager.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/NetworkConnectionManager.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/NetworkConnectionManager.java
deleted file mode 100644
index 6a5cc47..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/NetworkConnectionManager.java
+++ /dev/null
@@ -1,173 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-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;
-
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-/**
- * 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 ByteBufferedChannelManager byteBufferedChannelManager;
-
- public NetworkConnectionManager(final ByteBufferedChannelManager byteBufferedChannelManager,
- final InetAddress bindAddress, final int dataPort) throws IOException {
-
- final Configuration configuration = GlobalConfiguration.getConfiguration();
-
- this.byteBufferedChannelManager = byteBufferedChannelManager;
-
- // 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.byteBufferedChannelManager, 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 transferEnvelope
- * the envelope to be transfered
- */
- public void queueEnvelopeForTransfer(final RemoteReceiver remoteReceiver, final TransferEnvelope transferEnvelope) {
-
- getOutgoingConnection(remoteReceiver).queueEnvelope(transferEnvelope);
- }
-
- /**
- * 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());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnection.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnection.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnection.java
deleted file mode 100644
index 94463b6..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnection.java
+++ /dev/null
@@ -1,531 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-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;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.DefaultSerializer;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-/**
- * This class represents an outgoing TCP connection through which {@link TransferEnvelope} objects can be sent.
- * {@link TransferEnvelope} objects are received from the {@link ByteBufferedChannelManager} 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<TransferEnvelope> queuedEnvelopes = new ArrayDeque<TransferEnvelope>();
-
- /**
- * The {@link DefaultSerializer} object used to transform the envelopes into a byte stream.
- */
- private final DefaultSerializer serializer = new DefaultSerializer();
-
- /**
- * The {@link TransferEnvelope} that is currently processed.
- */
- private TransferEnvelope 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
-
- /**
- * 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;
- }
-
- /**
- * Adds a new {@link TransferEnvelope} 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 ByteBufferedChannelManager} object.
- *
- * @param transferEnvelope
- * the envelope to be added to the transfer queue
- */
- public void queueEnvelope(TransferEnvelope transferEnvelope) {
-
- synchronized (this.queuedEnvelopes) {
-
- checkConnection();
- this.queuedEnvelopes.add(transferEnvelope);
- }
- }
-
- 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<TransferEnvelope> iter = this.queuedEnvelopes.iterator();
- while (iter.hasNext()) {
- final TransferEnvelope 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 TransferEnvelope} 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.serializer.setTransferEnvelope(this.currentEnvelope);
- }
- }
- }
-
- if (!this.serializer.write(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 TransferEnvelope}
- * 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 TransferEnvelope} 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 TransferEnvelope} 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<TransferEnvelope> it = this.queuedEnvelopes.iterator();
- while (it.hasNext()) {
- final TransferEnvelope te = it.next();
- if (sourceChannelID.equals(te.getSource())) {
- number++;
- }
- }
-
- return number;
- }
- }
-
- /**
- * Removes all queued {@link TransferEnvelope} 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<TransferEnvelope> it = this.queuedEnvelopes.iterator();
- while (it.hasNext()) {
- final TransferEnvelope 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 ByteBufferedChannelManager} 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<TransferEnvelope> it = this.queuedEnvelopes.iterator();
- while (it.hasNext()) {
-
- final TransferEnvelope envelope = it.next();
- if (envelope.getBuffer() != null) {
- ++retVal;
- }
- }
- }
-
- return retVal;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java
deleted file mode 100644
index ef03a9c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutgoingConnectionThread.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-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() {
-
- 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));
- }
- }
-
- 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());
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java
deleted file mode 100644
index 7d8a571..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelContext.java
+++ /dev/null
@@ -1,17 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-public interface OutputChannelContext extends ChannelContext {
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java
deleted file mode 100644
index d9a16b7..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputChannelForwardingChain.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import java.io.IOException;
-import java.util.Queue;
-import java.util.concurrent.LinkedBlockingDeque;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-public final class OutputChannelForwardingChain {
-
- private final Queue<AbstractEvent> incomingEventQueue = new LinkedBlockingDeque<AbstractEvent>();
-
- private final AbstractOutputChannelForwarder first;
-
- private final AbstractOutputChannelForwarder last;
-
- public OutputChannelForwardingChain(final AbstractOutputChannelForwarder first,
- final AbstractOutputChannelForwarder last) {
-
- if (first == null) {
- throw new IllegalArgumentException("Argument first must not be null");
- }
-
- if (last == null) {
- throw new IllegalArgumentException("Argument last must not be null");
- }
-
- this.first = first;
- this.last = last;
- }
-
- public void pushEnvelope(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException {
-
- this.first.push(transferEnvelope);
- }
-
- public TransferEnvelope pullEnvelope() {
-
- return this.last.pull();
- }
-
- public void processEvent(final AbstractEvent event) {
-
- this.first.processEvent(event);
- }
-
- public boolean anyForwarderHasDataLeft() throws IOException, InterruptedException {
-
- return this.first.hasDataLeft();
- }
-
- public void destroy() {
-
- this.first.destroy();
- }
-
- public void processQueuedEvents() {
-
- AbstractEvent event = this.incomingEventQueue.poll();
- while (event != null) {
-
- this.first.processEvent(event);
- event = this.incomingEventQueue.poll();
- }
- }
-
- void offerEvent(final AbstractEvent event) {
- this.incomingEventQueue.offer(event);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.java
deleted file mode 100644
index fb2022a..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/OutputGateContext.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-
-public interface OutputGateContext extends GateContext {
-
- OutputChannelContext createOutputChannelContext(ChannelID channelID, OutputChannelContext previousContext,
- boolean isReceiverRunning, boolean mergeSpillBuffers);
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ReceiverNotFoundEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ReceiverNotFoundEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ReceiverNotFoundEvent.java
deleted file mode 100644
index 0b8f351..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/ReceiverNotFoundEvent.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.EventList;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-/**
- * An unknown receiver event can be used by the framework to inform a sender task that the delivery of a
- * {@link TransferEnvelope} has failed since the receiver could not be found.
- *
- */
-public final class ReceiverNotFoundEvent extends AbstractEvent {
-
- /**
- * The sequence number that will be set for transfer envelopes which contain the receiver not found event.
- */
- private static final int RECEIVER_NOT_FOUND_SEQUENCE_NUMBER = 0;
-
- /**
- * The ID of the receiver which could not be found
- */
- private ChannelID receiverID;
-
- /**
- * The sequence number of the envelope this event refers to
- */
- private int sequenceNumber;
-
- /**
- * Constructs a new unknown receiver event.
- *
- * @param receiverID
- * the ID of the receiver which could not be found
- * @param sequenceNumber
- * the sequence number of the envelope this event refers to
- */
- public ReceiverNotFoundEvent(final ChannelID receiverID, final int sequenceNumber) {
-
- if (receiverID == null) {
- throw new IllegalArgumentException("Argument unknownReceiverID must not be null");
- }
-
- if (sequenceNumber < 0) {
- throw new IllegalArgumentException("Argument sequenceNumber must be non-negative");
- }
-
- this.receiverID = receiverID;
- this.sequenceNumber = sequenceNumber;
- }
-
- /**
- * Default constructor for serialization/deserialization.
- */
- public ReceiverNotFoundEvent() {
-
- this.receiverID = new ChannelID();
- }
-
- /**
- * Returns the ID of the receiver which could not be found.
- *
- * @return the ID of the receiver which could not be found
- */
- public ChannelID getReceiverID() {
-
- return this.receiverID;
- }
-
- /**
- * Returns the sequence number of the envelope this event refers to.
- *
- * @return the sequence number of the envelope this event refers to
- */
- public int getSequenceNumber() {
-
- return this.sequenceNumber;
- }
-
-
- @Override
- public void write(final DataOutput out) throws IOException {
-
- this.receiverID.write(out);
- out.writeInt(this.sequenceNumber);
- }
-
-
- @Override
- public void read(final DataInput in) throws IOException {
-
- this.receiverID.read(in);
- this.sequenceNumber = in.readInt();
- }
-
- /**
- * Creates a transfer envelope which only contains a ReceiverNotFoundEvent.
- *
- * @param jobID
- * the ID of the job the event relates to.
- * @param receiver
- * the channel ID of the receiver that could not be found
- * @param sequenceNumber
- * the sequence number of the transfer envelope which caused the creation of this event
- * @return a transfer envelope which only contains a ReceiverNotFoundEvent
- */
- public static TransferEnvelope createEnvelopeWithEvent(final JobID jobID, final ChannelID receiver,
- final int sequenceNumber) {
-
- final TransferEnvelope transferEnvelope = new TransferEnvelope(RECEIVER_NOT_FOUND_SEQUENCE_NUMBER, jobID,
- receiver);
-
- final ReceiverNotFoundEvent unknownReceiverEvent = new ReceiverNotFoundEvent(receiver, sequenceNumber);
- transferEnvelope.addEvent(unknownReceiverEvent);
-
- return transferEnvelope;
- }
-
- /**
- * Checks if the given envelope only contains a ReceiverNotFoundEvent.
- *
- * @param transferEnvelope
- * the envelope to be checked
- * @return <code>true</code> if the envelope only contains a ReceiverNotFoundEvent, <code>false</code> otherwise
- */
- public static boolean isReceiverNotFoundEvent(final TransferEnvelope transferEnvelope) {
-
- if (transferEnvelope.getSequenceNumber() != RECEIVER_NOT_FOUND_SEQUENCE_NUMBER) {
- return false;
- }
-
- if (transferEnvelope.getBuffer() != null) {
- return false;
- }
-
- final EventList eventList = transferEnvelope.getEventList();
- if (eventList == null) {
- return false;
- }
-
- if (eventList.size() != 1) {
- return false;
- }
-
- if (!(eventList.get(0) instanceof ReceiverNotFoundEvent)) {
- return false;
- }
-
- return true;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RemoteReceiver.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RemoteReceiver.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RemoteReceiver.java
deleted file mode 100644
index db4e412..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/RemoteReceiver.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.util.StringUtils;
-
-/**
- * Objects of this class uniquely identify a connection to a remote {@link TaskManager}.
- *
- */
-public final class RemoteReceiver implements IOReadableWritable {
-
- /**
- * The address of the connection to the remote {@link TaskManager}.
- */
- private InetSocketAddress connectionAddress;
-
- /**
- * The index of the connection to the remote {@link TaskManager}.
- */
- private int connectionIndex;
-
- /**
- * Constructs a new remote receiver object.
- *
- * @param connectionAddress
- * the address of the connection to the remote {@link TaskManager}
- * @param connectionIndex
- * the index of the connection to the remote {@link TaskManager}
- */
- public RemoteReceiver(final InetSocketAddress connectionAddress, final int connectionIndex) {
-
- if (connectionAddress == null) {
- throw new IllegalArgumentException("Argument connectionAddress must not be null");
- }
-
- if (connectionIndex < 0) {
- throw new IllegalArgumentException("Argument connectionIndex must be a non-negative integer number");
- }
-
- this.connectionAddress = connectionAddress;
- this.connectionIndex = connectionIndex;
- }
-
- /**
- * Default constructor for serialization/deserialization.
- */
- public RemoteReceiver() {
- this.connectionAddress = null;
- this.connectionIndex = -1;
- }
-
- /**
- * Returns the address of the connection to the remote {@link TaskManager}.
- *
- * @return the address of the connection to the remote {@link TaskManager}
- */
- public InetSocketAddress getConnectionAddress() {
-
- return this.connectionAddress;
- }
-
- /**
- * Returns the index of the connection to the remote {@link TaskManager}.
- *
- * @return the index of the connection to the remote {@link TaskManager}
- */
- public int getConnectionIndex() {
-
- return this.connectionIndex;
- }
-
-
- @Override
- public int hashCode() {
-
- return this.connectionAddress.hashCode() + (31 * this.connectionIndex);
- }
-
-
- @Override
- public boolean equals(final Object obj) {
-
- if (!(obj instanceof RemoteReceiver)) {
- return false;
- }
-
- final RemoteReceiver rr = (RemoteReceiver) obj;
- if (!this.connectionAddress.equals(rr.connectionAddress)) {
- return false;
- }
-
- if (this.connectionIndex != rr.connectionIndex) {
- return false;
- }
-
- return true;
- }
-
-
- @Override
- public void write(final DataOutput out) throws IOException {
-
- final InetAddress ia = this.connectionAddress.getAddress();
- out.writeInt(ia.getAddress().length);
- out.write(ia.getAddress());
- out.writeInt(this.connectionAddress.getPort());
-
- out.writeInt(this.connectionIndex);
- }
-
-
- @Override
- public void read(final DataInput in) throws IOException {
-
- final int addr_length = in.readInt();
- final byte[] address = new byte[addr_length];
- in.readFully(address);
-
- InetAddress ia = null;
- try {
- ia = InetAddress.getByAddress(address);
- } catch (UnknownHostException uhe) {
- throw new IOException(StringUtils.stringifyException(uhe));
- }
- final int port = in.readInt();
- this.connectionAddress = new InetSocketAddress(ia, port);
-
- this.connectionIndex = in.readInt();
- }
-
-
- @Override
- public String toString() {
-
- return this.connectionAddress + " (" + this.connectionIndex + ")";
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/SenderHintEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/SenderHintEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/SenderHintEvent.java
deleted file mode 100644
index a6aebb1..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/SenderHintEvent.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.EventList;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-public final class SenderHintEvent extends AbstractEvent {
-
- /**
- * The sequence number that will be set for transfer envelopes which contain the sender hint event.
- */
- private static final int SENDER_HINT_SEQUENCE_NUMBER = 0;
-
- private final ChannelID source;
-
- private final RemoteReceiver remoteReceiver;
-
- SenderHintEvent(final ChannelID source, final RemoteReceiver remoteReceiver) {
-
- if (source == null) {
- throw new IllegalArgumentException("Argument source must not be null");
- }
-
- if (remoteReceiver == null) {
- throw new IllegalArgumentException("Argument remoteReceiver must not be null");
- }
-
- this.source = source;
- this.remoteReceiver = remoteReceiver;
- }
-
- public SenderHintEvent() {
-
- this.source = new ChannelID();
- this.remoteReceiver = new RemoteReceiver();
- }
-
- public ChannelID getSource() {
-
- return this.source;
- }
-
- public RemoteReceiver getRemoteReceiver() {
-
- return this.remoteReceiver;
- }
-
-
- @Override
- public void write(final DataOutput out) throws IOException {
-
- this.source.write(out);
- this.remoteReceiver.write(out);
- }
-
-
- @Override
- public void read(final DataInput in) throws IOException {
-
- this.source.read(in);
- this.remoteReceiver.read(in);
- }
-
- public static TransferEnvelope createEnvelopeWithEvent(final TransferEnvelope originalEnvelope,
- final ChannelID source, final RemoteReceiver remoteReceiver) {
-
- final TransferEnvelope transferEnvelope = new TransferEnvelope(SENDER_HINT_SEQUENCE_NUMBER,
- originalEnvelope.getJobID(), originalEnvelope.getSource());
-
- final SenderHintEvent senderEvent = new SenderHintEvent(source, remoteReceiver);
- transferEnvelope.addEvent(senderEvent);
-
- return transferEnvelope;
- }
-
- static boolean isSenderHintEvent(final TransferEnvelope transferEnvelope) {
-
- if (transferEnvelope.getSequenceNumber() != SENDER_HINT_SEQUENCE_NUMBER) {
- return false;
- }
-
- if (transferEnvelope.getBuffer() != null) {
- return false;
- }
-
- final EventList eventList = transferEnvelope.getEventList();
- if (eventList == null) {
- return false;
- }
-
- if (eventList.size() != 1) {
- return false;
- }
-
- if (!(eventList.get(0) instanceof SenderHintEvent)) {
- return false;
- }
-
- return true;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java
deleted file mode 100644
index 6c41a4f..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/TaskContext.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner;
-
-public interface TaskContext extends LocalBufferPoolOwner {
-
- OutputGateContext createOutputGateContext(GateID gateID);
-
- InputGateContext createInputGateContext(GateID gateID);
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnexpectedEnvelopeEvent.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnexpectedEnvelopeEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnexpectedEnvelopeEvent.java
deleted file mode 100644
index b7d59c6..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/bytebuffered/UnexpectedEnvelopeEvent.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/***********************************************************************************************************************
- * 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.bytebuffered;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-
-/**
- * This event is sent by an {@link InputChannelContext}. It indicates that the input channel context has received a
- * {@link TransferEnvelope} with a lower sequence number than expected. The typical reason for this is that data is
- * being replayed from a checkpoint. With the help of this event it is possible to request the sender to skip sending
- * transfer envelopes up to the given expected sequence number.
- *
- */
-public final class UnexpectedEnvelopeEvent extends AbstractEvent {
-
- /**
- * The expected sequence number.
- */
- private int expectedSequenceNumber;
-
- /**
- * Constructs a new unexpected envelope event.
- *
- * @param expectedSequenceNumber
- * the expected sequence number
- */
- public UnexpectedEnvelopeEvent(final int expectedSequenceNumber) {
-
- if (expectedSequenceNumber < 0) {
- throw new IllegalArgumentException("Argument expectedSequenceNumber must be non-negative.");
- }
-
- this.expectedSequenceNumber = expectedSequenceNumber;
- }
-
- /**
- * Default constructor for serialization/deserialization.
- */
- public UnexpectedEnvelopeEvent() {
- }
-
- /**
- * Returns the expected sequence number.
- *
- * @return the expected sequence number
- */
- public int getExpectedSequenceNumber() {
-
- return this.expectedSequenceNumber;
- }
-
-
- @Override
- public void write(final DataOutput out) throws IOException {
-
- out.writeInt(this.expectedSequenceNumber);
- }
-
-
- @Override
- public void read(final DataInput in) throws IOException {
-
- this.expectedSequenceNumber = in.readInt();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ExecutorThreadFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ExecutorThreadFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ExecutorThreadFactory.java
deleted file mode 100644
index ed845e8..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ExecutorThreadFactory.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/***********************************************************************************************************************
- * 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;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java
deleted file mode 100644
index 4a104d7..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/ForwardingBarrier.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.UnexpectedEnvelopeEvent;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-public final class ForwardingBarrier extends AbstractOutputChannelForwarder {
-
- private static final Log LOG = LogFactory.getLog(ForwardingBarrier.class);
-
- private final ChannelID outputChannelID;
-
- private int forwardingBarrier = -1;
-
- public ForwardingBarrier(final ChannelID outputChannelID, final AbstractOutputChannelForwarder next) {
- super(next);
-
- if (next == null) {
- throw new IllegalArgumentException("Argument next must not be null");
- }
-
- this.outputChannelID = outputChannelID;
- }
-
-
- @Override
- public void push(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException {
-
- if (transferEnvelope.getSequenceNumber() < this.forwardingBarrier) {
- recycleTransferEnvelope(transferEnvelope);
- return;
- }
-
- getNext().push(transferEnvelope);
- }
-
-
- @Override
- public void processEvent(final AbstractEvent event) {
-
- if (event instanceof UnexpectedEnvelopeEvent) {
-
- final UnexpectedEnvelopeEvent uee = (UnexpectedEnvelopeEvent) event;
- if (uee.getExpectedSequenceNumber() > this.forwardingBarrier) {
- this.forwardingBarrier = uee.getExpectedSequenceNumber();
- if (LOG.isDebugEnabled()) {
- LOG.debug("Setting forwarding barrier to sequence number " + this.forwardingBarrier
- + " for output channel " + this.outputChannelID);
- }
- }
- }
-
- getNext().processEvent(event);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java
deleted file mode 100644
index 5f14743..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeDispatcher.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.IOException;
-
-import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher;
-
-public final class RuntimeDispatcher extends AbstractOutputChannelForwarder {
-
- private final TransferEnvelopeDispatcher dispatcher;
-
- public RuntimeDispatcher(final TransferEnvelopeDispatcher dispatcher) {
- super(null);
-
- this.dispatcher = dispatcher;
- }
-
-
- @Override
- public void push(final TransferEnvelope transferEnvelope) throws IOException, InterruptedException {
-
- this.dispatcher.processEnvelopeFromOutputChannel(transferEnvelope);
- }
-}
[08/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java
index 56c2a8e..c9323a6 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java
@@ -42,6 +42,7 @@ import eu.stratosphere.pact.runtime.task.GroupReduceDriver;
import eu.stratosphere.pact.runtime.task.chaining.ChainedCollectorMapDriver;
import eu.stratosphere.pact.runtime.task.util.LocalStrategy;
import eu.stratosphere.pact.runtime.task.util.TaskConfig;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.test.iterative.nephele.JobGraphUtils;
import eu.stratosphere.test.recordJobs.kmeans.KMeansBroadcast.PointBuilder;
import eu.stratosphere.test.recordJobs.kmeans.KMeansBroadcast.RecomputeClusterCenter;
@@ -303,8 +304,6 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
JobGraphUtils.connect(head, output, ChannelType.NETWORK, DistributionPattern.POINTWISE);
JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
-
-
// -- instance sharing -------------------------------------------------------------------------------------
points.setVertexToShareInstancesWith(output);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/JobGraphUtils.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/JobGraphUtils.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/JobGraphUtils.java
index bac36bd..a229086 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/JobGraphUtils.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/JobGraphUtils.java
@@ -22,8 +22,8 @@ import eu.stratosphere.api.common.io.InputFormat;
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.nephele.client.JobClient;
import eu.stratosphere.nephele.client.JobExecutionException;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.nephele.jobgraph.AbstractJobVertex;
import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
index 906ccc0..06badd9 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
@@ -20,8 +20,8 @@ import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
import eu.stratosphere.api.java.record.io.FileOutputFormat;
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobInputVertex;
import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
@@ -285,7 +285,7 @@ public class CustomCompensatableDanglingPageRank {
JobGraphUtils.connect(pageWithRankInput, head, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
- JobGraphUtils.connect(head, intermediate, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ JobGraphUtils.connect(head, intermediate, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
JobGraphUtils.connect(adjacencyListInput, intermediate, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
@@ -295,8 +295,8 @@ public class CustomCompensatableDanglingPageRank {
tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, degreeOfParallelism);
- JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
- JobGraphUtils.connect(tail, fakeTailOutput, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
+ JobGraphUtils.connect(tail, fakeTailOutput, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
index 6edcef3..3e51808 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
@@ -20,8 +20,8 @@ import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
import eu.stratosphere.api.java.record.io.FileOutputFormat;
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobInputVertex;
import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
@@ -297,7 +297,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
JobGraphUtils.connect(pageWithRankInput, head, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
- JobGraphUtils.connect(head, intermediate, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ JobGraphUtils.connect(head, intermediate, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
JobGraphUtils.connect(adjacencyListInput, intermediate, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
@@ -307,8 +307,8 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, degreeOfParallelism);
- JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
- JobGraphUtils.connect(tail, fakeTailOutput, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
+ JobGraphUtils.connect(tail, fakeTailOutput, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
index ccf529a..b50c33e 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
@@ -19,8 +19,8 @@ import eu.stratosphere.api.common.typeutils.TypePairComparatorFactory;
import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
import eu.stratosphere.api.java.record.io.FileOutputFormat;
import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobInputVertex;
import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
@@ -265,7 +265,7 @@ public class CompensatableDanglingPageRank {
JobGraphUtils.connect(pageWithRankInput, head, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
- JobGraphUtils.connect(head, intermediate, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ JobGraphUtils.connect(head, intermediate, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
intermediateConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
JobGraphUtils.connect(adjacencyListInput, intermediate, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
@@ -275,8 +275,8 @@ public class CompensatableDanglingPageRank {
tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, 1);
tailConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(1, degreeOfParallelism);
- JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
- JobGraphUtils.connect(tail, fakeTailOutput, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
+ JobGraphUtils.connect(tail, fakeTailOutput, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilterTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilterTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilterTest.java
index 03e442e..04767c6 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilterTest.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/relational/query1Util/LineItemFilterTest.java
@@ -21,12 +21,12 @@ import static org.mockito.MockitoAnnotations.initMocks;
import java.util.ArrayList;
import java.util.List;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import org.junit.Before;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mock;
-import eu.stratosphere.nephele.io.AbstractRecordWriter;
import eu.stratosphere.pact.runtime.shipping.RecordOutputCollector;
import eu.stratosphere.test.recordJobs.util.Tuple;
import eu.stratosphere.types.IntValue;
@@ -39,9 +39,9 @@ public class LineItemFilterTest {
private static final String RETURN_FLAG = "N";
@Mock
- AbstractRecordWriter<Record> recordWriterMock;
+ RecordWriter<Record> recordWriterMock;
- private List<AbstractRecordWriter<Record>> writerList = new ArrayList<AbstractRecordWriter<Record>>();
+ private List<RecordWriter<Record>> writerList = new ArrayList<RecordWriter<Record>>();
@Before
public void setUp()
[23/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java
deleted file mode 100644
index 9d50e9c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractRecordWriter.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.event.task.EventListener;
-import eu.stratosphere.nephele.execution.Environment;
-import eu.stratosphere.nephele.io.channels.bytebuffered.EndOfSuperstepEvent;
-import eu.stratosphere.nephele.template.AbstractInvokable;
-
-/**
- * Abstract base class for a regular record writer and broadcast record writer.
- *
- * @param <T> The type of the record that can be emitted with this record writer.
- */
-public abstract class AbstractRecordWriter<T extends IOReadableWritable> implements Writer<T> {
-
- /**
- * The output gate assigned to this record writer.
- */
- private OutputGate<T> outputGate;
-
- /**
- * The environment associated to this record writer.
- */
- private Environment environment;
-
- /**
- * Constructs a new record writer and registers a new output gate with the application's environment.
- *
- * @param invokable
- * the application that instantiated the record writer
- * @param outputClass
- * the class of records that can be emitted with this record writer
- * @param selector
- * the channel selector to be used to determine the output channel to be used for a record
- * @param isBroadcast
- * <code>true</code> if this record writer shall broadcast the records to all connected channels,
- * <code>false/<code> otherwise
- */
- public AbstractRecordWriter(AbstractInvokable invokable, Class<T> outputClass, ChannelSelector<T> selector, boolean isBroadcast) {
- this.environment = invokable.getEnvironment();
- connectOutputGate(outputClass, selector, isBroadcast);
- }
-
- /**
- * Connects a record writer to an output gate.
- *
- * @param outputClass
- * the class of the record that can be emitted with this record writer
- * @param selector
- * the channel selector to be used to determine the output channel to be used for a record
- * @param isBroadcast
- * <code>true</code> if this record writer shall broadcast the records to all connected channels,
- * <code>false/<code> otherwise
- */
- private void connectOutputGate(Class<T> outputClass, ChannelSelector<T> selector, boolean isBroadcast)
- {
- GateID gateID = this.environment.getNextUnboundOutputGateID();
- if (gateID == null) {
- gateID = new GateID();
- }
-
- this.outputGate = this.environment.createOutputGate(gateID, outputClass, selector, isBroadcast);
- this.environment.registerOutputGate(this.outputGate);
- }
-
- /**
- * This method emits a record to the corresponding output gate. The method may block
- * until the record was transfered via any of the connected channels.
- *
- * @param record
- * The record to be emitted.
- * @throws IOException
- * Thrown on an error that may happen during the transfer of the given record or a previous record.
- */
- public void emit(final T record) throws IOException, InterruptedException {
- this.outputGate.writeRecord(record);
- }
-
- /**
- * Subscribes the listener object to receive events of the given type.
- *
- * @param eventListener
- * the listener object to register
- * @param eventType
- * the type of event to register the listener for
- */
- public void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
- this.outputGate.subscribeToEvent(eventListener, eventType);
- }
-
- /**
- * Removes the subscription for events of the given type for the listener object.
- *
- * @param eventListener
- * the listener object to cancel the subscription for
- * @param eventType
- * the type of the event to cancel the subscription for
- */
- public void unsubscribeFromEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType) {
- this.outputGate.unsubscribeFromEvent(eventListener, eventType);
- }
-
- /**
- * Publishes an event.
- *
- * @param event
- * the event to be published
- * @throws IOException
- * thrown if an error occurs while transmitting the event
- * @throws InterruptedException
- * thrown if the thread is interrupted while waiting for the event to be published
- */
- public void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException {
- this.outputGate.publishEvent(event);
- }
-
- public void flush() throws IOException, InterruptedException {
- this.outputGate.flush();
- }
-
- public void sendEndOfSuperstep() throws IOException, InterruptedException {
- this.outputGate.publishEvent(EndOfSuperstepEvent.INSTANCE);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractSingleGateRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractSingleGateRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractSingleGateRecordReader.java
deleted file mode 100644
index 8def532..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractSingleGateRecordReader.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.execution.Environment;
-import eu.stratosphere.nephele.template.AbstractInvokable;
-
-/**
- * This is an abstract base class for a record reader, either dealing with mutable or immutable records.
- *
- * @param <T> The type of the record that can be read from this record reader.
- */
-public abstract class AbstractSingleGateRecordReader<T extends IOReadableWritable> extends AbstractRecordReader {
-
- /**
- * The input gate associated with the record reader.
- */
- protected final InputGate<T> inputGate;
-
- // --------------------------------------------------------------------------------------------
-
- protected AbstractSingleGateRecordReader(AbstractInvokable invokable, RecordDeserializerFactory<T> deserializerFactory, int inputGateID) {
- Environment environment = invokable.getEnvironment();
- GateID gateID = environment.getNextUnboundInputGateID();
- if (gateID == null) {
- gateID = new GateID();
- }
-
- this.inputGate = environment.createInputGate(gateID, deserializerFactory);
- environment.registerInputGate(this.inputGate);
- }
-
- /**
- * Returns the number of input channels wired to this reader's input gate.
- *
- * @return the number of input channels wired to this reader's input gate
- */
- public int getNumberOfInputChannels() {
- return this.inputGate.getNumberOfInputChannels();
- }
-
- /**
- * Publishes an event.
- *
- * @param event
- * the event to be published
- * @throws IOException
- * thrown if an error occurs while transmitting the event
- * @throws InterruptedException
- * thrown if the thread is interrupted while waiting for the event to be published
- */
- @Override
- public void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException {
- // Delegate call to input gate to send events
- this.inputGate.publishEvent(event);
- }
-
-
- InputGate<T> getInputGate() {
- return this.inputGate;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractUnionRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractUnionRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractUnionRecordReader.java
deleted file mode 100644
index 5de3f67..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/AbstractUnionRecordReader.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.HashSet;
-import java.util.Set;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-
-public abstract class AbstractUnionRecordReader<T extends IOReadableWritable> extends AbstractRecordReader implements RecordAvailabilityListener<T> {
-
- /**
- * The set of all input gates.
- */
- private final InputGate<T>[] allInputGates;
-
- /**
- * The set of unclosed input gates.
- */
- private final Set<InputGate<T>> remainingInputGates;
-
- /**
- * Queue with indices of channels that store at least one available record.
- */
- private final ArrayDeque<InputGate<T>> availableInputGates = new ArrayDeque<InputGate<T>>();
-
- /**
- * The next input gate to read a record from.
- */
- private InputGate<T> nextInputGateToReadFrom;
-
-
- @Override
- public boolean isInputClosed() {
- return this.remainingInputGates.isEmpty();
- }
-
- /**
- * Constructs a new mutable union record reader.
- *
- * @param recordReaders
- * the individual mutable record readers whose input is used to construct the union
- */
- @SuppressWarnings("unchecked")
- protected AbstractUnionRecordReader(MutableRecordReader<T>[] recordReaders) {
-
- if (recordReaders == null) {
- throw new IllegalArgumentException("Provided argument recordReaders is null");
- }
-
- if (recordReaders.length < 2) {
- throw new IllegalArgumentException(
- "The mutable union record reader must at least be initialized with two individual mutable record readers");
- }
-
- this.allInputGates = new InputGate[recordReaders.length];
- this.remainingInputGates = new HashSet<InputGate<T>>((int) (recordReaders.length * 1.6f));
-
- for (int i = 0; i < recordReaders.length; i++) {
- InputGate<T> inputGate = recordReaders[i].getInputGate();
- inputGate.registerRecordAvailabilityListener(this);
- this.allInputGates[i] = inputGate;
- this.remainingInputGates.add(inputGate);
- }
- }
-
-
- @Override
- public void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException {
- for (InputGate<T> gate : this.allInputGates) {
- gate.publishEvent(event);
- }
- }
-
- @Override
- public void reportRecordAvailability(InputGate<T> inputGate) {
- synchronized (this.availableInputGates) {
- this.availableInputGates.add(inputGate);
- this.availableInputGates.notifyAll();
- }
- }
-
- protected boolean getNextRecord(T target) throws IOException, InterruptedException {
-
- while (true) {
- // has the current input gate more data?
- if (this.nextInputGateToReadFrom == null) {
- if (this.remainingInputGates.isEmpty()) {
- return false;
- }
-
- this.nextInputGateToReadFrom = getNextAvailableInputGate();
- }
-
- InputChannelResult result = this.nextInputGateToReadFrom.readRecord(target);
- switch (result) {
- case INTERMEDIATE_RECORD_FROM_BUFFER: // record is available and we can stay on the same channel
- return true;
-
- case LAST_RECORD_FROM_BUFFER: // record is available, but we need to re-check the channels
- this.nextInputGateToReadFrom = null;
- return true;
-
- case END_OF_SUPERSTEP:
- this.nextInputGateToReadFrom = null;
- if (incrementEndOfSuperstepEventAndCheck()) {
- return false; // end of the superstep
- }
- else {
- break; // fall through and wait for next record/event
- }
-
- case TASK_EVENT: // event for the subscribers is available
- handleEvent(this.nextInputGateToReadFrom.getCurrentEvent());
- this.nextInputGateToReadFrom = null;
- break;
-
- case END_OF_STREAM: // one gate is empty
- this.remainingInputGates.remove(this.nextInputGateToReadFrom);
- this.nextInputGateToReadFrom = null;
- break;
-
- case NONE: // gate processed an internal event and could not return a record on this call
- this.nextInputGateToReadFrom = null;
- break;
- }
- }
- }
-
- private InputGate<T> getNextAvailableInputGate() throws InterruptedException {
- synchronized (this.availableInputGates) {
- while (this.availableInputGates.isEmpty()) {
- this.availableInputGates.wait();
- }
- return this.availableInputGates.pop();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/BroadcastRecordWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/BroadcastRecordWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/BroadcastRecordWriter.java
deleted file mode 100644
index f2b1141..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/BroadcastRecordWriter.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.template.AbstractInputTask;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-/**
- * A record writer connects the application to an output gate. It allows the application
- * of emit (send out) to the output gate. The broadcast record writer will make sure that each emitted record will be
- * transfered via all connected output channels.
- *
- * @param <T>
- * the type of the record that can be emitted with this record writer
- */
-public class BroadcastRecordWriter<T extends IOReadableWritable> extends AbstractRecordWriter<T> {
-
- /**
- * Constructs a new broadcast record writer and registers a new output gate with the application's environment.
- *
- * @param taskBase
- * the application that instantiated the record writer
- * @param outputClass
- * the class of records that can be emitted with this record writer
- */
- public BroadcastRecordWriter(AbstractTask taskBase, Class<T> outputClass) {
- super(taskBase, outputClass, null, true);
- }
-
- /**
- * Constructs a new broadcast record writer and registers a new output gate with the application's environment.
- *
- * @param inputBase
- * the application that instantiated the record writer
- * @param outputClass
- * the class of records that can be emitted with this record writer
- */
- public BroadcastRecordWriter(AbstractInputTask<?> inputBase, Class<T> outputClass) {
- super(inputBase, outputClass, null, true);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ChannelSelector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ChannelSelector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ChannelSelector.java
deleted file mode 100644
index f7a7e9b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ChannelSelector.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- * Objects implementing this interface are passed to an {@link OutputGate}. When a record is sent through the output
- * gate, the channel selector object is called to determine to which {@link AbstractOutputChannel} objects the record
- * shall be passed on.
- *
- * @param <T>
- * the type of record which is sent through the attached output gate
- */
-public interface ChannelSelector<T extends IOReadableWritable> {
-
- /**
- * Called to determine to which attached {@link AbstractOutputChannel} objects the given record shall be forwarded.
- *
- * @param record
- * the record to the determine the output channels for
- * @param numberOfOutputChannels
- * the total number of output channels which are attached to respective output gate
- * @return a (possibly empty) array of integer numbers which indicate the indices of the output channels through
- * which the record shall be forwarded
- */
- int[] selectChannels(T record, int numberOfOutputChannels);
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DataOutputBuffer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DataOutputBuffer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DataOutputBuffer.java
deleted file mode 100644
index 171e985..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DataOutputBuffer.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/***********************************************************************************************************************
- * 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.
- **********************************************************************************************************************/
-
-/**
- * This file is based on source code from the Hadoop Project (http://hadoop.apache.org/), licensed by the Apache
- * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for
- * additional information regarding copyright ownership.
- */
-
-package eu.stratosphere.nephele.io;
-
-import java.io.DataInput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-
-/**
- * A reusable {@link DataOutput} implementation that writes to an in-memory
- * buffer.
- * <p>
- * This saves memory over creating a new DataOutputStream and ByteArrayOutputStream each time data is written.
- * <p>
- * Typical usage is something like the following:
- *
- * <pre>
- *
- * DataOutputBuffer buffer = new DataOutputBuffer();
- * while (... loop condition ...) {
- * buffer.reset();
- * ... write buffer using DataOutput methods ...
- * byte[] data = buffer.getData();
- * int dataLength = buffer.getLength();
- * ... write data to its ultimate destination ...
- * }
- * </pre>
- */
-public class DataOutputBuffer extends DataOutputStream {
-
- private static class ByteBufferedOutputStream extends OutputStream {
-
- private ByteBuffer buf;
-
- public ByteBuffer getData() {
- return this.buf;
- }
-
- public int getLength() {
- return this.buf.limit();
- }
-
- public ByteBufferedOutputStream() {
- this(1024);
- }
-
- public ByteBufferedOutputStream(int size) {
- this.buf = ByteBuffer.allocate(size);
- this.buf.position(0);
- this.buf.limit(0);
- }
-
- public void reset() {
- this.buf.position(0);
- this.buf.limit(0);
- }
-
- public void write(DataInput in, int len) throws IOException {
-
- final int newcount = this.buf.limit() + len;
- if (newcount > this.buf.capacity()) {
- final ByteBuffer newBuf = ByteBuffer.allocate(Math.max(this.buf.capacity() << 1, newcount));
- newBuf.position(0);
- System.arraycopy(this.buf.array(), 0, newBuf.array(), 0, this.buf.limit());
- newBuf.limit(this.buf.limit());
- this.buf = newBuf;
- }
-
- in.readFully(this.buf.array(), this.buf.limit(), len);
- this.buf.limit(newcount);
- }
-
- @Override
- public void write(byte[] b, int off, int len) throws IOException {
-
- final int newcount = this.buf.limit() + len;
- if (newcount > this.buf.capacity()) {
- increaseInternalBuffer(newcount);
- }
-
- System.arraycopy(b, off, this.buf.array(), this.buf.limit(), len);
- this.buf.limit(newcount);
- }
-
- @Override
- public void write(byte[] b) throws IOException {
- write(b, 0, b.length);
- }
-
- @Override
- public void write(int arg0) throws IOException {
-
- final int oldLimit = this.buf.limit();
- final int newLimit = oldLimit + 1;
-
- if (newLimit > this.buf.capacity()) {
- increaseInternalBuffer(newLimit);
- }
-
- this.buf.limit(newLimit);
- this.buf.put(oldLimit, (byte) arg0);
- }
-
- private void increaseInternalBuffer(int minimumRequiredSize) {
- final ByteBuffer newBuf = ByteBuffer.allocate(Math.max(this.buf.capacity() << 1, minimumRequiredSize));
- newBuf.position(0);
- System.arraycopy(this.buf.array(), 0, newBuf.array(), 0, this.buf.limit());
- newBuf.limit(this.buf.limit());
- this.buf = newBuf;
- }
- }
-
- private final ByteBufferedOutputStream byteBufferedOutputStream;
-
- /** Constructs a new empty buffer. */
- public DataOutputBuffer() {
- this(new ByteBufferedOutputStream());
- }
-
- public DataOutputBuffer(int size) {
- this(new ByteBufferedOutputStream(size));
- }
-
- private DataOutputBuffer(ByteBufferedOutputStream byteBufferedOutputStream) {
- super(byteBufferedOutputStream);
- this.byteBufferedOutputStream = byteBufferedOutputStream;
- }
-
- public ByteBuffer getData() {
- return this.byteBufferedOutputStream.getData();
- }
-
- public int getLength() {
- return this.byteBufferedOutputStream.getLength();
- }
-
- /** Resets the buffer to empty. */
- public DataOutputBuffer reset() {
- this.byteBufferedOutputStream.reset();
- return this;
- }
-
- public void write(DataInput in, int length) throws IOException {
- this.byteBufferedOutputStream.write(in, length);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DefaultChannelSelector.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DefaultChannelSelector.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DefaultChannelSelector.java
deleted file mode 100644
index cabc208..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DefaultChannelSelector.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- * This is the default implementation of the {@link ChannelSelector} interface. It represents a simple round-robin
- * strategy, i.e. regardless of the record every attached exactly one output channel is selected at a time.
-
- * @param <T>
- * the type of record which is sent through the attached output gate
- */
-public class DefaultChannelSelector<T extends IOReadableWritable> implements ChannelSelector<T> {
-
- /**
- * Stores the index of the channel to send the next record to.
- */
- private final int[] nextChannelToSendTo = new int[1];
-
- /**
- * Constructs a new default channel selector.
- */
- public DefaultChannelSelector() {
- this.nextChannelToSendTo[0] = 0;
- }
-
-
- @Override
- public int[] selectChannels(final T record, final int numberOfOutpuChannels) {
-
- this.nextChannelToSendTo[0] = (this.nextChannelToSendTo[0] + 1) % numberOfOutpuChannels;
-
- return this.nextChannelToSendTo;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DistributionPattern.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DistributionPattern.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DistributionPattern.java
deleted file mode 100644
index 634fbcc..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/DistributionPattern.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-/**
- * A distribution pattern determines which subtasks of a producing Nephele task a wired to which
- * subtasks of a consuming subtask.
- *
- */
-
-public enum DistributionPattern {
-
- /**
- * Each subtask of the producing Nephele task is wired to each subtask of the consuming Nephele task.
- */
- BIPARTITE,
-
- /**
- * The i-th subtask of the producing Nephele task is wired to the i-th subtask of the consuming Nephele task.
- */
- POINTWISE
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Gate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Gate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Gate.java
deleted file mode 100644
index 26773ca..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Gate.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.event.task.EventListener;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-public interface Gate<T extends IOReadableWritable> {
-
- /**
- * Returns the index that has been assigned to the gate upon initialization.
- *
- * @return the index that has been assigned to the gate upon initialization.
- */
- int getIndex();
-
- /**
- * Subscribes the listener object to receive events of the given type.
- *
- * @param eventListener
- * the listener object to register
- * @param eventType
- * the type of event to register the listener for
- */
- void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType);
-
- /**
- * Removes the subscription for events of the given type for the listener object.
- *
- * @param eventListener
- * the listener object to cancel the subscription for
- * @param eventType
- * the type of the event to cancel the subscription for
- */
- void unsubscribeFromEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType);
-
- /**
- * Publishes an event.
- *
- * @param event
- * the event to be published
- * @throws IOException
- * thrown if an error occurs while transmitting the event
- * @throws InterruptedException
- * thrown if the thread is interrupted while waiting for the event to be published
- */
- void publishEvent(AbstractEvent event) throws IOException, InterruptedException;
-
- /**
- * Passes a received event on to the event notification manager so it cam ne dispatched.
- *
- * @param event
- * the event to pass on to the notification manager
- */
- void deliverEvent(AbstractTaskEvent event);
-
- /**
- * Returns the ID of the job this gate belongs to.
- *
- * @return the ID of the job this gate belongs to
- */
- JobID getJobID();
-
- /**
- * Returns the type of the input/output channels which are connected to this gate.
- *
- * @return the type of input/output channels which are connected to this gate
- */
- ChannelType getChannelType();
-
- /**
- * Returns the ID of the gate.
- *
- * @return the ID of the gate
- */
- GateID getGateID();
-
- /**
- * Releases the allocated resources (particularly buffer) of all channels attached to this gate. This method
- * should only be called after the respected task has stopped running.
- */
- void releaseAllChannelResources();
-
- /**
- * Checks if the gate is closed. The gate is closed if all this associated channels are closed.
- *
- * @return <code>true</code> if the gate is closed, <code>false</code> otherwise
- * @throws IOException
- * thrown if any error occurred while closing the gate
- * @throws InterruptedException
- * thrown if the gate is interrupted while waiting for this operation to complete
- */
- boolean isClosed() throws IOException, InterruptedException;
-
- /**
- * Checks if the considered gate is an input gate.
- *
- * @return <code>true</code> if the considered gate is an input gate, <code>false</code> if it is an output gate
- */
- boolean isInputGate();
-
- /**
- * Sets the type of the input/output channels which are connected to this gate.
- *
- * @param channelType
- * the type of input/output channels which are connected to this gate
- */
- void setChannelType(ChannelType channelType);
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/GateID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/GateID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/GateID.java
deleted file mode 100644
index 9998916..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/GateID.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-/**
- * A class for statistically unique gate IDs.
- *
- */
-public final class GateID extends AbstractID {
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ImmutableRecordDeserializerFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ImmutableRecordDeserializerFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ImmutableRecordDeserializerFactory.java
deleted file mode 100644
index be0a60f..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ImmutableRecordDeserializerFactory.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.channels.DefaultDeserializer;
-
-/**
- * As simple factory implementation that instantiates deserializers for immutable records. For
- * each deserialization, a new record is instantiated from the given class.
- */
-public class ImmutableRecordDeserializerFactory<T extends IOReadableWritable> implements RecordDeserializerFactory<T> {
-
- private final Class<? extends T> recordType; // the type of the record to be deserialized
-
-
- /**
- * Creates a new factory that instantiates deserializers for immutable records.
- *
- * @param recordType The type of the record to be deserialized.
- */
- public ImmutableRecordDeserializerFactory(final Class<? extends T> recordType) {
- this.recordType = recordType;
- }
-
- @Override
- public RecordDeserializer<T> createDeserializer() {
- return new DefaultDeserializer<T>(this.recordType);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputChannelResult.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputChannelResult.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputChannelResult.java
deleted file mode 100644
index f154a3f..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputChannelResult.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-public enum InputChannelResult {
-
- NONE,
- INTERMEDIATE_RECORD_FROM_BUFFER,
- LAST_RECORD_FROM_BUFFER,
- END_OF_SUPERSTEP,
- TASK_EVENT,
- END_OF_STREAM;
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputGate.java
deleted file mode 100644
index 6a57756..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/InputGate.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.io.channels.AbstractInputChannel;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryInputChannel;
-import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkInputChannel;
-
-/**
- * @param <T> The type of record that can be transported through this gate.
- */
-public interface InputGate<T extends IOReadableWritable> extends Gate<T> {
-
- /**
- * Reads a record from one of the associated input channels. Channels are read such that one buffer from a channel is
- * consecutively consumed. The buffers in turn are consumed in the order in which they arrive.
- * Note that this method is not guaranteed to return a record, because the currently available channel data may not always
- * constitute an entire record, when events or partial records are part of the data.
- *
- * When called even though no data is available, this call will block until data is available, so this method should be called
- * when waiting is desired (such as when synchronously consuming a single gate) or only when it is known that data is available
- * (such as when reading a union of multiple input gates).
- *
- * @param target The record object into which to construct the complete record.
- * @return The result indicating whether a complete record is available, a event is available, only incomplete data
- * is available (NONE), or the gate is exhausted.
- * @throws IOException Thrown when an error occurred in the network stack relating to this channel.
- * @throws InterruptedException Thrown, when the thread working on this channel is interrupted.
- */
- InputChannelResult readRecord(T target) throws IOException, InterruptedException;
-
- /**
- * Returns the number of input channels associated with this input gate.
- *
- * @return the number of input channels associated with this input gate
- */
- int getNumberOfInputChannels();
-
- /**
- * Returns the input channel from position <code>pos</code> of the gate's internal channel list.
- *
- * @param pos
- * the position to retrieve the channel from
- * @return the channel from the given position or <code>null</code> if such position does not exist.
- */
- AbstractInputChannel<T> getInputChannel(int pos);
-
- /**
- * Notify the gate that the channel with the given index has
- * at least one record available.
- *
- * @param channelIndex
- * the index of the channel which has at least one record available
- */
- void notifyRecordIsAvailable(int channelIndex);
-
- /**
- * Notify the gate that is has consumed a data unit from the channel with the given index
- *
- * @param channelIndex
- * the index of the channel from which a data unit has been consumed
- */
- void notifyDataUnitConsumed(int channelIndex);
-
- /**
- * Immediately closes the input gate and all its input channels. The corresponding
- * output channels are notified. Any remaining records in any buffers or queue is considered
- * irrelevant and is discarded.
- *
- * @throws IOException
- * thrown if an I/O error occurs while closing the gate
- * @throws InterruptedException
- * thrown if the thread is interrupted while waiting for the gate to be closed
- */
- void close() throws IOException, InterruptedException;
-
- /**
- * Creates a new network input channel and assigns it to the given input gate.
- *
- * @param inputGate
- * the input gate the channel shall be assigned to
- * @param channelID
- * the ID of the channel
- * @param connectedChannelID
- * the ID of the channel this channel is connected to
- * @param compressionLevel
- * the level of compression to be used for this channel
- * @return the new network input channel
- */
- NetworkInputChannel<T> createNetworkInputChannel(InputGate<T> inputGate, ChannelID channelID,
- ChannelID connectedChannelID);
-
-
- /**
- * Creates a new in-memory input channel and assigns it to the given input gate.
- *
- * @param inputGate
- * the input gate the channel shall be assigned to
- * @param channelID
- * the ID of the channel
- * @param connectedChannelID
- * the ID of the channel this channel is connected to
- * @param compressionLevel
- * the level of compression to be used for this channel
- * @return the new in-memory input channel
- */
- InMemoryInputChannel<T> createInMemoryInputChannel(InputGate<T> inputGate, ChannelID channelID,
- ChannelID connectedChannelID);
-
- /**
- * Registers a {@link RecordAvailabilityListener} with this input gate.
- *
- * @param listener
- * the listener object to be registered
- */
- void registerRecordAvailabilityListener(RecordAvailabilityListener<T> listener);
-
-
- AbstractTaskEvent getCurrentEvent();
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableReader.java
deleted file mode 100644
index ef1080c..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableReader.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- *
- */
-public interface MutableReader<T extends IOReadableWritable> extends ReaderBase {
-
- /**
- * @param target
- * @return
- * @throws IOException
- * @throws InterruptedException
- */
- boolean next(T target) throws IOException, InterruptedException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordDeserializerFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordDeserializerFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordDeserializerFactory.java
deleted file mode 100644
index 5890562..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordDeserializerFactory.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.channels.DefaultDeserializer;
-
-/**
- * As simple factory implementation that instantiates deserializers for mutable records.
- */
-public class MutableRecordDeserializerFactory<T extends IOReadableWritable> implements RecordDeserializerFactory<T> {
-
- /**
- * Creates a new factory that instantiates deserializers for immutable records.
- *
- * @param recordType The type of the record to be deserialized.
- */
- public MutableRecordDeserializerFactory() {}
-
- @Override
- public RecordDeserializer<T> createDeserializer() {
- return new DefaultDeserializer<T>(null);
- }
-
- // --------------------------------------------------------------------------------------------
-
- private static final RecordDeserializerFactory<IOReadableWritable> INSTANCE =
- new MutableRecordDeserializerFactory<IOReadableWritable>();
-
- /**
- * Gets the singleton instance of the {@code MutableRecordDeserializerFactory}.
- *
- * @param <E> The generic type of the record to be deserialized.
- * @return An instance of the factory.
- */
- public static final <E extends IOReadableWritable> RecordDeserializerFactory<E> get() {
- @SuppressWarnings("unchecked")
- RecordDeserializerFactory<E> toReturn = (RecordDeserializerFactory<E>) INSTANCE;
- return toReturn;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordReader.java
deleted file mode 100644
index 23c26c4..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableRecordReader.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-public class MutableRecordReader<T extends IOReadableWritable> extends AbstractSingleGateRecordReader<T> implements MutableReader<T> {
-
- private boolean endOfStream;
-
-
- /**
- * Constructs a new mutable record reader and registers a new input gate with the application's environment.
- *
- * @param taskBase The application that instantiated the record reader.
- */
- public MutableRecordReader(final AbstractTask taskBase) {
- super(taskBase, MutableRecordDeserializerFactory.<T>get(), 0);
- }
-
- /**
- * Constructs a new record reader and registers a new input gate with the application's environment.
- *
- * @param outputBase The application that instantiated the record reader.
- */
- public MutableRecordReader(final AbstractOutputTask outputBase) {
- super(outputBase, MutableRecordDeserializerFactory.<T>get(), 0);
- }
-
- /**
- * Constructs a new record reader and registers a new input gate with the application's environment.
- *
- * @param taskBase
- * the application that instantiated the record reader
- * @param inputGateID
- * The ID of the input gate that the reader reads from.
- */
- public MutableRecordReader(final AbstractTask taskBase, final int inputGateID) {
- super(taskBase, MutableRecordDeserializerFactory.<T>get(), inputGateID);
- }
-
- /**
- * Constructs a new record reader and registers a new input gate with the application's environment.
- *
- * @param outputBase
- * the application that instantiated the record reader
- * @param inputGateID
- * The ID of the input gate that the reader reads from.
- */
- public MutableRecordReader(final AbstractOutputTask outputBase, final int inputGateID) {
- super(outputBase, MutableRecordDeserializerFactory.<T>get(), inputGateID);
- }
-
- // --------------------------------------------------------------------------------------------
-
- @Override
- public boolean next(final T target) throws IOException, InterruptedException {
- if (this.endOfStream) {
- return false;
-
- }
- while (true) {
- InputChannelResult result = this.inputGate.readRecord(target);
- switch (result) {
- case INTERMEDIATE_RECORD_FROM_BUFFER:
- case LAST_RECORD_FROM_BUFFER:
- return true;
-
- case END_OF_SUPERSTEP:
- if (incrementEndOfSuperstepEventAndCheck()) {
- return false; // end of the superstep
- }
- else {
- break; // fall through and wait for next record/event
- }
-
- case TASK_EVENT:
- handleEvent(this.inputGate.getCurrentEvent());
- break; // fall through to get next record
-
- case END_OF_STREAM:
- this.endOfStream = true;
- return false;
-
- default:
- ; // fall through to get next record
- }
- }
- }
-
- @Override
- public boolean isInputClosed() {
- return this.endOfStream;
- }
-
- @Override
- public void setIterative(int numEventsUntilEndOfSuperstep) {
- // sanity check for debug purposes
- if (numEventsUntilEndOfSuperstep != getNumberOfInputChannels()) {
- throw new IllegalArgumentException("Number of events till end of superstep is different from the number of input channels.");
- }
- super.setIterative(numEventsUntilEndOfSuperstep);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableUnionRecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableUnionRecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableUnionRecordReader.java
deleted file mode 100644
index 1fc8dd2..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/MutableUnionRecordReader.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-public class MutableUnionRecordReader<T extends IOReadableWritable> extends AbstractUnionRecordReader<T> implements MutableReader<T> {
-
-
- /**
- * Constructs a new mutable union record reader.
- *
- * @param recordReaders
- * the individual mutable record readers whose input is used to construct the union
- */
- public MutableUnionRecordReader(MutableRecordReader<T>[] recordReaders) {
- super(recordReaders);
- }
-
- @Override
- public boolean next(T target) throws IOException, InterruptedException {
- return getNextRecord(target);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/OutputGate.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/OutputGate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/OutputGate.java
deleted file mode 100644
index 0bbe5e0..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/OutputGate.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-import java.util.List;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.channels.AbstractOutputChannel;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.bytebuffered.InMemoryOutputChannel;
-import eu.stratosphere.nephele.io.channels.bytebuffered.NetworkOutputChannel;
-
-/**
- * In Nephele output gates are a specialization of general gates and connect
- * record writers and output channels. As channels, output gates are always
- * parameterized to a specific type of record which they can transport.
- *
- * @param <T>
- * the type of record that can be transported through this gate
- */
-public interface OutputGate<T extends IOReadableWritable> extends Gate<T> {
-
- /**
- * Returns the type of record that can be transported through this gate.
- *
- * @return the type of record that can be transported through this gate
- */
- Class<T> getType();
-
- /**
- * Writes a record to one of the associated output channels. Currently, the
- * channels are chosen in a simple round-robin fashion. This operation may
- * block until the respective channel has received the data.
- *
- * @param record
- * the record to be written
- * @throws IOException
- * thrown if any error occurs during channel I/O
- */
- void writeRecord(T record) throws IOException, InterruptedException;
-
- /**
- * Returns all the OutputChannels connected to this gate
- *
- * @return the list of OutputChannels connected to this RecordWriter
- */
- List<AbstractOutputChannel<T>> getOutputChannels();
-
- /**
- * Flushes all connected output channels.
- *
- * @throws IOException
- * thrown if an error occurs while flushing an output channel
- * @throws InterruptedException
- * thrown if the thread is interrupted while waiting for the data to be flushed
- */
- void flush() throws IOException, InterruptedException;
-
- /**
- * Checks if this output gate operates in broadcast mode, i.e. all records passed to it are transferred through all
- * connected output channels.
- *
- * @return <code>true</code> if this output gate operates in broadcast mode, <code>false</code> otherwise
- */
- boolean isBroadcast();
-
- /**
- * Returns the number of output channels associated with this output gate.
- *
- * @return the number of output channels associated with this output gate
- */
- int getNumberOfOutputChannels();
-
- /**
- * Returns the output channel from position <code>pos</code> of the gate's
- * internal channel list.
- *
- * @param pos
- * the position to retrieve the channel from
- * @return the channel from the given position or <code>null</code> if such
- * position does not exist.
- */
- AbstractOutputChannel<T> getOutputChannel(int pos);
-
- /**
- * Returns the output gate's channel selector.
- *
- * @return the output gate's channel selector or <code>null</code> if the gate operates in broadcast mode
- */
- ChannelSelector<T> getChannelSelector();
-
- /**
- * Requests the output gate to closed. This means the application will send
- * no records through this gate anymore.
- *
- * @throws IOException
- * @throws InterruptedException
- */
- void requestClose() throws IOException, InterruptedException;
-
- /**
- * Removes all output channels from the output gate.
- */
- void removeAllOutputChannels();
-
- /**
- * Creates a new network output channel and assigns it to the given output gate.
- *
- * @param outputGate
- * the output gate the channel shall be assigned to
- * @param channelID
- * the ID of the channel
- * @param connectedChannelID
- * the ID of the channel this channel is connected to
- * @param compressionLevel
- * the level of compression to be used for this channel
- * @return the new network output channel
- */
- NetworkOutputChannel<T> createNetworkOutputChannel(OutputGate<T> outputGate, ChannelID channelID,
- ChannelID connectedChannelID);
-
- /**
- * Creates a new in-memory output channel and assigns it to the given output gate.
- *
- * @param outputGate
- * the output gate the channel shall be assigned to
- * @param channelID
- * the ID of the channel
- * @param connectedChannelID
- * the ID of the channel this channel is connected to
- * @param compressionLevel
- * the level of compression to be used for this channel
- * @return the new in-memory output channel
- */
- InMemoryOutputChannel<T> createInMemoryOutputChannel(OutputGate<T> outputGate, ChannelID channelID,
- ChannelID connectedChannelID);
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Reader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Reader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Reader.java
deleted file mode 100644
index 80d2010..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/Reader.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- * A reader interface to read records from an input.
- *
- * @param <T> The type of the record that can be emitted with this record writer
- */
-public interface Reader<T extends IOReadableWritable> extends ReaderBase {
-
- boolean hasNext() throws IOException, InterruptedException;
-
- T next() throws IOException, InterruptedException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ReaderBase.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ReaderBase.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ReaderBase.java
deleted file mode 100644
index ae69ad0..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/ReaderBase.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.event.task.EventListener;
-
-
-/**
- *
- */
-public interface ReaderBase {
-
- boolean isInputClosed();
-
- /**
- * Subscribes the listener object to receive events of the given type.
- *
- * @param eventListener
- * the listener object to register
- * @param eventType
- * the type of event to register the listener for
- */
- void subscribeToEvent(EventListener eventListener, Class<? extends AbstractTaskEvent> eventType);
-
- /**
- * Removes the subscription for events of the given type for the listener object.
- *
- * @param eventListener
- * the listener object to cancel the subscription for
- * @param eventType
- * the type of the event to cancel the subscription for
- */
- void unsubscribeFromEvent(final EventListener eventListener, final Class<? extends AbstractTaskEvent> eventType);
-
- /**
- * Publishes an event.
- *
- * @param event
- * the event to be published
- * @throws IOException
- * thrown if an error occurs while transmitting the event
- * @throws InterruptedException
- * thrown if the thread is interrupted while waiting for the event to be published
- */
- void publishEvent(AbstractTaskEvent event) throws IOException, InterruptedException;
-
-
- void setIterative(int numEventsUntilEndOfSuperstep);
-
-
- void startNextSuperstep();
-
- boolean hasReachedEndOfSuperstep();
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordAvailabilityListener.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordAvailabilityListener.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordAvailabilityListener.java
deleted file mode 100644
index d22f048..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordAvailabilityListener.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- * This interface can be implemented by a class which shall be notified by an input gate when one of the its connected
- * input channels has at least one record available for reading.
- *
- * @param <T>
- * the type of record transported through the corresponding input gate
- */
-public interface RecordAvailabilityListener<T extends IOReadableWritable> {
-
- /**
- * This method is called by an input gate when one of its connected input channels has at least one record available
- * for reading.
- *
- * @param inputGate
- * the input gate which has at least one record available
- */
- void reportRecordAvailability(InputGate<T> inputGate);
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializer.java
deleted file mode 100644
index e8ccb77..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializer.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-import java.nio.channels.ReadableByteChannel;
-
-/**
- * This interface must be implemented by classes which transfer bytes streams back into {@link Record} objects.
- *
- * @param <T> The type of record this record deserializer works with.
- */
-public interface RecordDeserializer<T>
-{
- /**
- * Transforms a record back from a readable byte channel. The deserialization might not complete, because the channel
- * has not all required data available. In that case, this method must return {@code null}. Furthermore, it may
- * not retain a reference to the given target object in that case, but must manage to put the data aside.
- *
- * @param target The record object into which to deserialize the data. May be null for deserializers
- * that operate on immutable objects, in which case the deserializer has to instantiate an
- * object. In the case where this object is non-null, but the deserialization does not complete,
- * the object must not be used to cache the partial state, as it is not guaranteed that the object
- * will remain unchanged until the next attempt to continue the deserialization.
- * @param in The byte stream which contains the record's data.
- * @return The record deserialized from <code>in</code>, or null, if the record .
- * @throws IOException Thrown if an I/O error occurs while deserializing the record from the stream
- */
- T readData(final T target, final ReadableByteChannel readableByteChannel) throws IOException;
-
- /**
- * Clears the internal buffers of the deserializer and resets its state.
- */
- void clear();
-
- /**
- * Checks whether the deserializer has data from a previous deserialization attempt stored in its internal buffers which
- * is not yet finished.
- *
- * @return <code>true</code>, if the deserializer's internal buffers contain data from a previous deserialization
- * attempt, <code>false</code> otherwise.
- */
- boolean hasUnfinishedData();
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializerFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializerFactory.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializerFactory.java
deleted file mode 100644
index eadd326..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordDeserializerFactory.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-/**
- * A simple factory to instantiate record deserializer objects. Since a deserializer might be stateful, the system
- * must be able to instantiate an arbitrary number of them, equal to the number of data channels.
- *
- * If the created deserializers are in fact not stateful, the factory should return a shared object.
- */
-public interface RecordDeserializerFactory<T>
-{
- /**
- * Creates a new instance of the deserializer. The returned instance may not share any state with
- * any previously returned instance.
- *
- * @return An instance of the deserializer.
- */
- RecordDeserializer<T> createDeserializer();
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordReader.java
deleted file mode 100644
index 5010013..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordReader.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import java.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-/**
- * A record writer connects an input gate to an application. It allows the application
- * query for incoming records and read them from input gate.
- *
- * @param <T> The type of the record that can be read from this record reader.
- */
-public class RecordReader<T extends IOReadableWritable> extends AbstractSingleGateRecordReader<T> implements Reader<T> {
-
- private final Class<T> recordType;
-
- /**
- * Stores the last read record.
- */
- private T lookahead;
-
- /**
- * Stores if more no more records will be received from the assigned input gate.
- */
- private boolean noMoreRecordsWillFollow;
-
- // --------------------------------------------------------------------------------------------
-
- /**
- * Constructs a new record reader and registers a new input gate with the application's environment.
- *
- * @param taskBase
- * The application that instantiated the record reader.
- * @param recordType
- * The class of records that can be read from the record reader.
- */
- public RecordReader(AbstractTask taskBase, Class<T> recordType) {
- super(taskBase, MutableRecordDeserializerFactory.<T>get(), 0);
- this.recordType = recordType;
- }
-
- /**
- * Constructs a new record reader and registers a new input gate with the application's environment.
- *
- * @param outputBase
- * The application that instantiated the record reader.
- * @param recordType
- * The class of records that can be read from the record reader.
- */
- public RecordReader(AbstractOutputTask outputBase, Class<T> recordType) {
- super(outputBase, MutableRecordDeserializerFactory.<T>get(), 0);
- this.recordType = recordType;
- }
-
- // --------------------------------------------------------------------------------------------
-
- /**
- * Checks if at least one more record can be read from the associated input gate. This method may block
- * until the associated input gate is able to read the record from one of its input channels.
- *
- * @return <code>true</code>it at least one more record can be read from the associated input gate, otherwise
- * <code>false</code>
- */
- @Override
- public boolean hasNext() throws IOException, InterruptedException{
- if (this.lookahead != null) {
- return true;
- } else {
- if (this.noMoreRecordsWillFollow) {
- return false;
- }
-
- T record = instantiateRecordType();
-
- while (true) {
- InputChannelResult result = this.inputGate.readRecord(record);
- switch (result) {
- case INTERMEDIATE_RECORD_FROM_BUFFER:
- case LAST_RECORD_FROM_BUFFER:
- this.lookahead = record;
- return true;
-
- case END_OF_SUPERSTEP:
- if (incrementEndOfSuperstepEventAndCheck()) {
- return false;
- }
- else {
- break; // fall through and wait for next record/event
- }
-
- case TASK_EVENT:
- handleEvent(this.inputGate.getCurrentEvent());
- break;
-
- case END_OF_STREAM:
- this.noMoreRecordsWillFollow = true;
- return false;
-
- default:
- ; // fall through the loop
- }
- }
- }
- }
-
- /**
- * Reads the current record from the associated input gate.
- *
- * @return the current record from the associated input gate.
- * @throws IOException
- * thrown if any error occurs while reading the record from the input gate
- */
- @Override
- public T next() throws IOException, InterruptedException {
- if (hasNext()) {
- T tmp = this.lookahead;
- this.lookahead = null;
- return tmp;
- } else {
- return null;
- }
- }
-
- @Override
- public boolean isInputClosed() {
- return this.noMoreRecordsWillFollow;
- }
-
- private T instantiateRecordType() {
- try {
- return this.recordType.newInstance();
- } catch (InstantiationException e) {
- throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
- } catch (IllegalAccessException e) {
- throw new RuntimeException("Cannot instantiate class '" + this.recordType.getName() + "'.", e);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordWriter.java
deleted file mode 100644
index 2f4c21b..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/io/RecordWriter.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.template.AbstractInputTask;
-import eu.stratosphere.nephele.template.AbstractTask;
-
-/**
- * A record writer connects the application to an output gate. It allows the application
- * of emit (send out) to the output gate. The output gate will then take care of distributing
- * the emitted records among the output channels.
- *
- * @param <T>
- * the type of the record that can be emitted with this record writer
- */
-public class RecordWriter<T extends IOReadableWritable> extends AbstractRecordWriter<T> {
-
- /**
- * Constructs a new record writer and registers a new output gate with the application's environment.
- *
- * @param taskBase
- * the application that instantiated the record writer
- * @param outputClass
- * the class of records that can be emitted with this record writer
- * @param selector
- * the channel selector to be used to determine the output channel to be used for a record
- */
- public RecordWriter(AbstractTask taskBase, Class<T> outputClass, ChannelSelector<T> selector) {
- super(taskBase, outputClass, selector, false);
- }
-
- /**
- * Constructs a new record writer and registers a new output gate with the application's environment.
- *
- * @param taskBase
- * the application that instantiated the record writer
- * @param outputClass
- * the class of records that can be emitted with this record writer
- */
- public RecordWriter(AbstractTask taskBase, Class<T> outputClass) {
- super(taskBase, outputClass, null, false);
- }
-
- /**
- * This method emits a record to the corresponding output gate. The method may block
- * until the record was transfered via any of the connected channels.
- *
- * @param inputBase
- * the application that instantiated the record writer
- * @param outputClass
- * the class of records that can be emitted with this record writer
- */
- public RecordWriter(AbstractInputTask<?> inputBase, Class<T> outputClass) {
- super(inputBase, outputClass, null, false);
- }
-
- /**
- * Constructs a new record writer and registers a new output gate with the application's environment.
- *
- * @param inputBase
- * the application that instantiated the record writer
- * @param outputClass
- * the class of records that can be emitted with this record writer
- * @param selector
- * the channel selector to be used to determine the output channel to be used for a record
- */
- public RecordWriter(AbstractInputTask<?> inputBase, Class<T> outputClass, ChannelSelector<T> selector) {
- super(inputBase, outputClass, selector, false);
- }
-}
[13/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProviderBroker.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProviderBroker.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProviderBroker.java
new file mode 100644
index 0000000..518fe47
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProviderBroker.java
@@ -0,0 +1,24 @@
+/***********************************************************************************************************************
+ * 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.bufferprovider;
+
+import java.io.IOException;
+
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.nephele.jobgraph.JobID;
+
+public interface BufferProviderBroker {
+
+ BufferProvider getBufferProvider(JobID jobID, ChannelID sourceChannelID) throws IOException;
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/GlobalBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/GlobalBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/GlobalBufferPool.java
new file mode 100644
index 0000000..2141017
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/GlobalBufferPool.java
@@ -0,0 +1,123 @@
+/***********************************************************************************************************************
+ * 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.bufferprovider;
+
+import java.util.Queue;
+import java.util.concurrent.ArrayBlockingQueue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import eu.stratosphere.core.memory.MemorySegment;
+
+/**
+ * A global buffer pool for the network stack.
+ * <p>
+ * All buffers used by the network stack come from this pool. Requests to this pool are mediated by instances of
+ * {@link LocalBufferPool}.
+ * <p>
+ * The size and number of buffers can be configured via the global system config.
+ */
+public final class GlobalBufferPool {
+
+ private final static Log LOG = LogFactory.getLog(GlobalBufferPool.class);
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ /** Total number of buffers */
+ private final int numBuffers;
+
+ /** Size of each buffer (in bytes) */
+ private final int bufferSize;
+
+ /** The available buffers */
+ private final Queue<MemorySegment> buffers;
+
+ private boolean isDestroyed;
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ public GlobalBufferPool(int numBuffers, int bufferSize) {
+ this.numBuffers = numBuffers;
+ this.bufferSize = bufferSize;
+
+ this.buffers = new ArrayBlockingQueue<MemorySegment>(this.numBuffers);
+ for (int i = 0; i < this.numBuffers; i++) {
+ this.buffers.add(new MemorySegment(new byte[this.bufferSize]));
+ }
+
+ LOG.info(String.format("Initialized global buffer pool with %d buffers (%d bytes each).",
+ this.numBuffers, this.bufferSize));
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ /**
+ * Requests a buffer <strong>from</strong> the pool.
+ *
+ * @return buffer from pool or <code>null</code>, if no buffer available
+ */
+ public MemorySegment requestBuffer() {
+ return this.buffers.poll();
+ }
+
+ /**
+ * Returns a buffer <em>to</em> the pool.
+ *
+ * @param buffer the buffer to be returned
+ */
+ public void returnBuffer(MemorySegment buffer) {
+ this.buffers.add(buffer);
+ }
+
+ /**
+ * Returns the size of buffers (in bytes).
+ *
+ * @return size of buffers (in bytes)
+ */
+ public int getBufferSize() {
+ return this.bufferSize;
+ }
+
+ /**
+ * Returns the total number of managed buffers.
+ *
+ * @return total number of managed buffers
+ */
+ public int numBuffers() {
+ return this.numBuffers;
+ }
+
+ /**
+ * Returns the number of currently available buffers.
+ *
+ * @return currently available number of buffers
+ */
+ public int numAvailableBuffers() {
+ return this.buffers.size();
+ }
+
+ public synchronized void destroy() {
+ if (!this.isDestroyed) {
+ // mark as shutdown and release memory
+ this.isDestroyed = true;
+
+ for (MemorySegment buffer : this.buffers) {
+ buffer.free();
+ }
+
+ this.buffers.clear();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
new file mode 100644
index 0000000..e8aeb11
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
@@ -0,0 +1,306 @@
+/***********************************************************************************************************************
+ * 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.bufferprovider;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.BufferRecycler;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+/**
+ * A buffer pool used to manage a designated number of buffers from a {@link GlobalBufferPool}.
+ * <p>
+ * A local buffer pool mediates buffer requests to the global buffer pool to ensure dead-lock free operation of the
+ * network stack by limiting the number of designated buffers per local buffer pool. It also implements the default
+ * mechanism for buffer recycling, which ensures that every buffer is ultimately returned to the global buffer pool.
+ */
+public final class LocalBufferPool implements BufferProvider {
+
+ private static final class LocalBufferPoolRecycler implements BufferRecycler {
+
+ private final LocalBufferPool bufferPool;
+
+ private LocalBufferPoolRecycler(LocalBufferPool bufferPool) {
+ this.bufferPool = bufferPool;
+ }
+
+ @Override
+ public void recycle(MemorySegment buffer) {
+ this.bufferPool.recycleBuffer(buffer);
+ }
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ /** Time (ms) to wait before retry for blocking buffer requests */
+ private static final int WAIT_TIME = 100;
+
+ /** Global buffer pool to request buffers from */
+ private final GlobalBufferPool globalBufferPool;
+
+ /** Buffers managed by this local buffer pool */
+ private final Queue<MemorySegment> buffers = new ArrayDeque<MemorySegment>();
+
+ /** The recycler via which to return buffers to this local buffer pool */
+ private final LocalBufferPoolRecycler recycler;
+
+ /** Queue of buffer availability listeners */
+ private final Queue<BufferAvailabilityListener> listeners = new ArrayDeque<BufferAvailabilityListener>();
+
+ /** Size of each buffer in this pool (in bytes) */
+ private final int bufferSize;
+
+ /** Number of buffers assigned to this local buffer pool */
+ private int numDesignatedBuffers;
+
+ /** Number of buffers requested from the global buffer pool */
+ private int numRequestedBuffers;
+
+ /** Flag to indicate whether an asynchronous event has been reported */
+ private boolean hasAsyncEventOccurred;
+
+ /** Flag to indicate whether this local buffer pool has been destroyed */
+ private boolean isDestroyed;
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ public LocalBufferPool(GlobalBufferPool globalBufferPool, int numDesignatedBuffers) {
+ this.globalBufferPool = globalBufferPool;
+ this.bufferSize = globalBufferPool.getBufferSize();
+ this.numDesignatedBuffers = numDesignatedBuffers;
+
+ this.recycler = new LocalBufferPoolRecycler(this);
+ }
+ // -----------------------------------------------------------------------------------------------------------------
+
+ @Override
+ public Buffer requestBuffer(int minBufferSize) throws IOException {
+ try {
+ return requestBuffer(minBufferSize, false);
+ } catch (InterruptedException e) {
+ throw new IOException("Unexpected InterruptedException while non-blocking buffer request.");
+ }
+ }
+
+ @Override
+ public Buffer requestBufferBlocking(int minBufferSize) throws IOException, InterruptedException {
+ return requestBuffer(minBufferSize, true);
+ }
+
+ /**
+ * Requests a buffer from this local buffer pool.
+ * <p>
+ * A non-blocking call to this method will only return a buffer, if one is available in the local pool after
+ * having returned excess buffers. Otherwise, it will return <code>null</code>.
+ * <p>
+ * A blocking call will request a new buffer from the global buffer and block until one is available or an
+ * asynchronous event has been reported via {@link #reportAsynchronousEvent()}.
+ *
+ * @param minBufferSize minimum size of requested buffer (in bytes)
+ * @param isBlocking flag to indicate whether to block until buffer is available
+ * @return buffer from the global buffer pool or <code>null</code>, if no buffer available
+ * @throws IOException
+ * @throws InterruptedException
+ */
+ private Buffer requestBuffer(int minBufferSize, boolean isBlocking) throws IOException, InterruptedException {
+ if (minBufferSize > this.bufferSize) {
+ throw new IllegalArgumentException(String.format("Too large buffer requested (requested %d, maximum %d).",
+ minBufferSize, this.bufferSize));
+ }
+
+ while (true) {
+ boolean isAsyncRequest = false;
+
+ synchronized (this.buffers) {
+ // Return excess buffers to global buffer pool
+ while (this.numRequestedBuffers > this.numDesignatedBuffers) {
+ final MemorySegment buffer = this.buffers.poll();
+ if (buffer == null) {
+ break;
+ }
+
+ this.globalBufferPool.returnBuffer(buffer);
+ this.numRequestedBuffers--;
+ }
+
+ // Request buffers from global buffer pool
+ while (this.buffers.isEmpty()) {
+ if (this.numRequestedBuffers < this.numDesignatedBuffers) {
+ final MemorySegment buffer = this.globalBufferPool.requestBuffer();
+
+ if (buffer != null) {
+ this.buffers.add(buffer);
+
+ this.numRequestedBuffers++;
+ continue;
+ }
+ }
+
+ if (this.hasAsyncEventOccurred && isBlocking) {
+ this.hasAsyncEventOccurred = false;
+ isAsyncRequest = true;
+ break;
+ }
+
+ if (isBlocking) {
+ this.buffers.wait(WAIT_TIME);
+ } else {
+ return null;
+ }
+ }
+
+ if (!isAsyncRequest) {
+ return new Buffer(this.buffers.poll(), minBufferSize, this.recycler);
+ }
+ }
+ }
+ }
+
+ @Override
+ public int getBufferSize() {
+ return this.bufferSize;
+ }
+
+ @Override
+ public void reportAsynchronousEvent() {
+ synchronized (this.buffers) {
+ this.hasAsyncEventOccurred = true;
+ this.buffers.notify();
+ }
+ }
+
+ @Override
+ public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+ synchronized (this.buffers) {
+ if (!this.buffers.isEmpty()) {
+ return false;
+ }
+
+ if (this.isDestroyed) {
+ return false;
+ }
+
+ this.listeners.add(listener);
+ }
+
+ return true;
+ }
+
+ /**
+ * Sets the designated number of buffers for this local buffer pool and returns excess buffers to the global buffer
+ * pool.
+ * <p>
+ * The designated number of buffers determines how many buffers this buffer pool is allowed to manage. New buffers
+ * can only be requested, if the requested number of buffers is less than the designated number. If possible, excess
+ * buffers will be returned to the global buffer pool.
+ *
+ * @param numDesignatedBuffers number of buffers designated for this local buffer pool
+ */
+ public void setNumDesignatedBuffers(int numDesignatedBuffers) {
+ synchronized (this.buffers) {
+ this.numDesignatedBuffers = numDesignatedBuffers;
+
+ // Return excess buffers to global buffer pool
+ while (this.numRequestedBuffers > this.numDesignatedBuffers) {
+ if (this.buffers.isEmpty()) {
+ break;
+ }
+
+ this.globalBufferPool.returnBuffer(this.buffers.poll());
+ this.numRequestedBuffers --;
+ }
+
+ this.buffers.notify();
+ }
+ }
+
+ /**
+ * Returns the number of buffers available in the local buffer pool.
+ *
+ * @return number of available buffers
+ */
+ public int numAvailableBuffers() {
+ synchronized (this.buffers) {
+ return this.buffers.size();
+ }
+ }
+
+ /**
+ * Returns the number of buffers, which have been requested from the global buffer pool.
+ *
+ * @return number of buffers requested from the global buffer pool
+ */
+ public int numRequestedBuffers() {
+ synchronized (this.buffers) {
+ return this.numRequestedBuffers;
+ }
+ }
+
+ /**
+ * Returns the designated number of buffers for this local buffer pool.
+ *
+ * @return number of designated buffers for this buffer pool
+ */
+ public int numDesignatedBuffers() {
+ synchronized (this.buffers) {
+ return this.numDesignatedBuffers;
+ }
+ }
+
+ /**
+ * Destroys this local buffer pool and immediately returns all available buffers to the global buffer pool.
+ * <p>
+ * Buffers, which have been requested from this local buffer pool via <code>requestBuffer</code> cannot be returned
+ * immediately and will be returned when the respective buffer is recycled (see {@link #recycleBuffer(MemorySegment)}).
+ */
+ public void destroy() {
+ synchronized (this.buffers) {
+ if (this.isDestroyed) {
+ return;
+ }
+
+ this.isDestroyed = true;
+
+ // return all buffers
+ while (!this.buffers.isEmpty()) {
+ this.globalBufferPool.returnBuffer(this.buffers.poll());
+ this.numRequestedBuffers--;
+ }
+ }
+ }
+
+ /**
+ * Returns a buffer to the buffer pool and notifies listeners about the availability of a new buffer.
+ *
+ * @param buffer buffer to return to the buffer pool
+ */
+ private void recycleBuffer(MemorySegment buffer) {
+ synchronized (this.buffers) {
+ if (this.isDestroyed) {
+ this.globalBufferPool.returnBuffer(buffer);
+ this.numRequestedBuffers--;
+ } else {
+ this.buffers.add(buffer);
+ this.buffers.notify();
+
+ while (!this.listeners.isEmpty()) {
+ this.listeners.poll().bufferAvailable();
+ }
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolOwner.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolOwner.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolOwner.java
new file mode 100644
index 0000000..ccab2ca
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolOwner.java
@@ -0,0 +1,56 @@
+/***********************************************************************************************************************
+ * 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.bufferprovider;
+
+/**
+ * A local buffer pool owner is an object which initially retrieves its buffers from the {@link GlobalBufferPool} and
+ * manages its fraction of the overall buffer pool locally by means of a {@link LocalBufferPool}.
+ *
+ */
+public interface LocalBufferPoolOwner {
+
+ /**
+ * Returns the number of byte-buffered channels that will retrieve their buffers from the local buffer pool.
+ *
+ * @return the number of byte-buffered channels that will retrieve their buffers from the local buffer pool
+ */
+ int getNumberOfChannels();
+
+ /**
+ * Sets the designated number of buffers the local buffer pool owner is allowed to fetch from the global buffer pool
+ * and manage locally by means of the {@link LocalBufferPool}.
+ *
+ * @param numBuffers
+ * the numBuffers the local buffer pool owner is allowed to fetch from the global buffer pool
+ */
+ void setDesignatedNumberOfBuffers(int numBuffers);
+
+ /**
+ * Clears the local buffer pool and returns all buffers to the global buffer pool.
+ */
+ void clearLocalBufferPool();
+
+ void registerGlobalBufferPool(GlobalBufferPool globalBufferPool);
+
+ /**
+ * Logs the current status of the local buffer pool. This method is intended mainly for debugging purposes.
+ */
+ void logBufferUtilization();
+
+ /**
+ * Reports an asynchronous event. Calling this method interrupts each blocking method of the buffer pool owner and
+ * allows the blocked thread to respond to the event.
+ */
+ void reportAsynchronousEvent();
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java
new file mode 100644
index 0000000..217a5ce
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/SerialSingleBufferPool.java
@@ -0,0 +1,77 @@
+/***********************************************************************************************************************
+ * 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.bufferprovider;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.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
+ public Buffer requestBuffer(int minBufferSize) {
+ if (minBufferSize <= this.bufferSize) {
+ return this.buffer.duplicate();
+ }
+ else {
+ throw new IllegalArgumentException("Requesting buffer with size " + minBufferSize + ". Pool's buffer size is " + this.bufferSize);
+ }
+ }
+
+ @Override
+ public Buffer requestBufferBlocking(int minBufferSize) {
+ if (minBufferSize <= this.bufferSize) {
+ return this.buffer.duplicate();
+ }
+ else {
+ throw new IllegalArgumentException("Requesting buffer with size " + minBufferSize + ". Pool's buffer size is " + this.bufferSize);
+ }
+ }
+
+ @Override
+ public int getBufferSize() {
+ return this.bufferSize;
+ }
+
+ @Override
+ public void reportAsynchronousEvent() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void recycle(MemorySegment buffer) {}
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java
new file mode 100644
index 0000000..a692aec
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/Envelope.java
@@ -0,0 +1,169 @@
+/***********************************************************************************************************************
+ * 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.event.task.AbstractEvent;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.nephele.jobgraph.JobID;
+import eu.stratosphere.runtime.io.serialization.DataInputDeserializer;
+import eu.stratosphere.runtime.io.serialization.DataOutputSerializer;
+import eu.stratosphere.util.InstantiationUtil;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public final class Envelope {
+
+ private final JobID jobID;
+
+ private final ChannelID source;
+
+ private final int sequenceNumber;
+
+ private ByteBuffer serializedEventList;
+
+ private Buffer buffer;
+
+ public Envelope(int sequenceNumber, JobID jobID, ChannelID source) {
+ this.sequenceNumber = sequenceNumber;
+ this.jobID = jobID;
+ this.source = source;
+ }
+
+ private Envelope(Envelope toDuplicate) {
+ this.jobID = toDuplicate.jobID;
+ this.source = toDuplicate.source;
+ this.sequenceNumber = toDuplicate.sequenceNumber;
+ this.serializedEventList = null;
+ this.buffer = null;
+ }
+
+ public Envelope duplicate() {
+ Envelope duplicate = new Envelope(this);
+ if (hasBuffer()) {
+ duplicate.setBuffer(this.buffer.duplicate());
+ }
+
+ return duplicate;
+ }
+
+ public Envelope duplicateWithoutBuffer() {
+ return new Envelope(this);
+ }
+
+ public JobID getJobID() {
+ return this.jobID;
+ }
+
+ public ChannelID getSource() {
+ return this.source;
+ }
+
+ public int getSequenceNumber() {
+ return this.sequenceNumber;
+ }
+
+ public void setEventsSerialized(ByteBuffer serializedEventList) {
+ if (this.serializedEventList != null)
+ throw new IllegalStateException("Event list has already been set.");
+
+ this.serializedEventList = serializedEventList;
+ }
+
+ public void serializeEventList(List<? extends AbstractEvent> eventList) {
+ if (this.serializedEventList != null)
+ throw new IllegalStateException("Event list has already been set.");
+
+ this.serializedEventList = serializeEvents(eventList);
+ }
+
+ public ByteBuffer getEventsSerialized() {
+ return this.serializedEventList;
+ }
+
+ public List<? extends AbstractEvent> deserializeEvents() {
+ return deserializeEvents(getClass().getClassLoader());
+ }
+
+ public List<? extends AbstractEvent> deserializeEvents(ClassLoader classloader) {
+ if (this.serializedEventList == null) {
+ return Collections.emptyList();
+ }
+
+ try {
+ DataInputDeserializer deserializer = new DataInputDeserializer(this.serializedEventList);
+
+ int numEvents = deserializer.readInt();
+ ArrayList<AbstractEvent> events = new ArrayList<AbstractEvent>(numEvents);
+
+ for (int i = 0; i < numEvents; i++) {
+ String className = deserializer.readUTF();
+ Class<? extends AbstractEvent> clazz;
+ try {
+ clazz = Class.forName(className).asSubclass(AbstractEvent.class);
+ } catch (ClassNotFoundException e) {
+ throw new RuntimeException("Could not load event class '" + className + "'.", e);
+ } catch (ClassCastException e) {
+ throw new RuntimeException("The class '" + className + "' is no valid subclass of '" + AbstractEvent.class.getName() + "'.", e);
+ }
+
+ AbstractEvent evt = InstantiationUtil.instantiate(clazz, AbstractEvent.class);
+ evt.read(deserializer);
+
+ events.add(evt);
+ }
+
+ return events;
+ }
+ catch (IOException e) {
+ throw new RuntimeException("Error while deserializing the events.", e);
+ }
+ }
+
+ public void setBuffer(Buffer buffer) {
+ this.buffer = buffer;
+ }
+
+ public Buffer getBuffer() {
+ return this.buffer;
+ }
+
+ private ByteBuffer serializeEvents(List<? extends AbstractEvent> events) {
+ try {
+ // create the serialized event list
+ DataOutputSerializer serializer = events.size() == 0
+ ? new DataOutputSerializer(4)
+ : new DataOutputSerializer(events.size() * 32);
+ serializer.writeInt(events.size());
+
+ for (AbstractEvent evt : events) {
+ serializer.writeUTF(evt.getClass().getName());
+ evt.write(serializer);
+ }
+
+ return serializer.wrapAsByteBuffer();
+ }
+ catch (IOException e) {
+ throw new RuntimeException("Error while serializing the task events.", e);
+ }
+ }
+
+ public boolean hasBuffer() {
+ return this.buffer != null;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java
new file mode 100644
index 0000000..2b69c0d
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeDispatcher.java
@@ -0,0 +1,46 @@
+/***********************************************************************************************************************
+ * 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 java.io.IOException;
+
+/**
+ * A envelope dispatcher receives {@link Envelope}s and sends them to all of its destinations.
+ */
+public interface EnvelopeDispatcher {
+
+ /**
+ * Dispatches an envelope from an output channel to the receiving input channels (forward flow).
+ *
+ * @param envelope envelope to be sent
+ */
+ void dispatchFromOutputChannel(Envelope envelope) throws IOException, InterruptedException;
+
+ /**
+ * Dispatches an envelope from an input channel to the receiving output channels (backwards flow).
+ *
+ * @param envelope envelope to be sent
+ */
+ void dispatchFromInputChannel(Envelope envelope) throws IOException, InterruptedException;
+
+ /**
+ * Dispatches an envelope from an incoming TCP connection.
+ * <p>
+ * After an envelope has been constructed from a TCP socket, this method is called to send the envelope to the
+ * receiving input channel.
+ *
+ * @param envelope envelope to be sent
+ */
+ void dispatchFromNetwork(Envelope envelope) throws IOException, InterruptedException;
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java
new file mode 100644
index 0000000..7b7e178
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReader.java
@@ -0,0 +1,212 @@
+/***********************************************************************************************************************
+ * 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;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.java
new file mode 100644
index 0000000..f99e1f2
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReceiverList.java
@@ -0,0 +1,75 @@
+/***********************************************************************************************************************
+ * 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 java.net.InetAddress;
+
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.network.ConnectionInfoLookupResponse;
+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
+ * receivers: Local receivers identified by {@link ChannelID} objects, remote receivers identified by
+ * {@link InetAddress} objects and finally checkpoints which are identified by
+ * <p>
+ * This class is thread-safe.
+ *
+ */
+public class EnvelopeReceiverList {
+
+ private final ChannelID localReceiver;
+
+ private final RemoteReceiver remoteReceiver;
+
+ public EnvelopeReceiverList(ConnectionInfoLookupResponse cilr) {
+ this.localReceiver = cilr.getLocalTarget();
+ this.remoteReceiver = cilr.getRemoteTarget();
+ }
+
+ public EnvelopeReceiverList(ChannelID localReceiver) {
+ this.localReceiver = localReceiver;
+ this.remoteReceiver = null;
+ }
+
+ public EnvelopeReceiverList(RemoteReceiver remoteReceiver) {
+ this.localReceiver = null;
+ this.remoteReceiver = remoteReceiver;
+ }
+
+ public boolean hasLocalReceiver() {
+ return this.localReceiver != null;
+ }
+
+ public boolean hasRemoteReceiver() {
+ return this.remoteReceiver != null;
+ }
+
+ public int getTotalNumberOfReceivers() {
+ return (this.localReceiver == null ? 0 : 1) + (this.remoteReceiver == null ? 0 : 1);
+ }
+
+ public RemoteReceiver getRemoteReceiver() {
+ return this.remoteReceiver;
+ }
+
+ public ChannelID getLocalReceiver() {
+ return this.localReceiver;
+ }
+
+ @Override
+ public String toString() {
+ return "local receiver: " + this.localReceiver + ", remote receiver: " + this.remoteReceiver;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java
new file mode 100644
index 0000000..c00e61b
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeWriter.java
@@ -0,0 +1,134 @@
+/***********************************************************************************************************************
+ * 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();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.java
new file mode 100644
index 0000000..f7d49bf
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/envelope/NoBufferAvailableException.java
@@ -0,0 +1,53 @@
+/***********************************************************************************************************************
+ * 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.runtime.io.network.bufferprovider.BufferProvider;
+
+/**
+ * This exception is thrown to indicate that the deserialization process of a {@link Envelope} could not be
+ * continued because a {@link Buffer} to store the envelope's content is currently not available.
+ *
+ */
+public final class NoBufferAvailableException extends Exception {
+
+ /**
+ * Generated serial UID.
+ */
+ 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
+ * the buffer provider which could not deliver a buffer
+ */
+ public NoBufferAvailableException(final BufferProvider bufferProvider) {
+ this.bufferProvider = bufferProvider;
+ }
+
+ /**
+ * Returns the buffer provider which could not deliver a buffer.
+ *
+ * @return the buffer provider which could not deliver a buffer
+ */
+ public BufferProvider getBufferProvider() {
+ return this.bufferProvider;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java
new file mode 100644
index 0000000..f22e6f7
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnection.java
@@ -0,0 +1,115 @@
+/***********************************************************************************************************************
+ * 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();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java
new file mode 100644
index 0000000..774ad4e
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/IncomingConnectionThread.java
@@ -0,0 +1,226 @@
+/***********************************************************************************************************************
+ * 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);
+ }
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java
new file mode 100644
index 0000000..7df1901
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnection.java
@@ -0,0 +1,529 @@
+/***********************************************************************************************************************
+ * 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;
+ }
+}
[12/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java
new file mode 100644
index 0000000..dde26e3
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/tcp/OutgoingConnectionThread.java
@@ -0,0 +1,276 @@
+/***********************************************************************************************************************
+ * 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());
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java
new file mode 100644
index 0000000..f4c8aec
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java
@@ -0,0 +1,521 @@
+/***********************************************************************************************************************
+ * 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.serialization;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+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.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+/**
+ * @param <T> The type of the record to be deserialized.
+ */
+public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> implements RecordDeserializer<T> {
+
+ private final NonSpanningWrapper nonSpanningWrapper;
+
+ private final SpanningWrapper spanningWrapper;
+
+ public AdaptiveSpanningRecordDeserializer() {
+ this.nonSpanningWrapper = new NonSpanningWrapper();
+ this.spanningWrapper = new SpanningWrapper();
+ }
+
+ @Override
+ public void setNextMemorySegment(MemorySegment segment, int numBytes) throws IOException {
+ // check if some spanning record deserialization is pending
+ if (this.spanningWrapper.getNumGatheredBytes() > 0) {
+ this.spanningWrapper.addNextChunkFromMemorySegment(segment, numBytes);
+ }
+ else {
+ this.nonSpanningWrapper.initializeFromMemorySegment(segment, 0, numBytes);
+ }
+ }
+
+ @Override
+ public DeserializationResult getNextRecord(T target) throws IOException {
+ // always check the non-spanning wrapper first.
+ // this should be the majority of the cases for small records
+ // for large records, this portion of the work is very small in comparison anyways
+
+ int nonSpanningRemaining = this.nonSpanningWrapper.remaining();
+
+ // check if we can get a full length;
+ if (nonSpanningRemaining >= 4) {
+ int len = this.nonSpanningWrapper.readInt();
+ if (len <= nonSpanningRemaining - 4) {
+ // we can get a full record from here
+ target.read(this.nonSpanningWrapper);
+
+ return (this.nonSpanningWrapper.remaining() == 0) ?
+ DeserializationResult.LAST_RECORD_FROM_BUFFER :
+ DeserializationResult.INTERMEDIATE_RECORD_FROM_BUFFER;
+ } else {
+ // we got the length, but we need the rest from the spanning deserializer
+ // and need to wait for more buffers
+ this.spanningWrapper.initializeWithPartialRecord(this.nonSpanningWrapper, len);
+ this.nonSpanningWrapper.clear();
+ return DeserializationResult.PARTIAL_RECORD;
+ }
+ } else if (nonSpanningRemaining > 0) {
+ // we have an incomplete length
+ // add our part of the length to the length buffer
+ this.spanningWrapper.initializeWithPartialLength(this.nonSpanningWrapper);
+ this.nonSpanningWrapper.clear();
+ return DeserializationResult.PARTIAL_RECORD;
+ }
+
+ // spanning record case
+ if (this.spanningWrapper.hasFullRecord()) {
+ // get the full record
+ target.read(this.spanningWrapper);
+
+ // move the remainder to the non-spanning wrapper
+ // this does not copy it, only sets the memory segment
+ this.spanningWrapper.moveRemainderToNonSpanningDeserializer(this.nonSpanningWrapper);
+ this.spanningWrapper.clear();
+
+ return (this.nonSpanningWrapper.remaining() == 0) ?
+ DeserializationResult.LAST_RECORD_FROM_BUFFER :
+ DeserializationResult.INTERMEDIATE_RECORD_FROM_BUFFER;
+// } else if (this.spanningWrapper.getNumGatheredBytes() == 0) {
+// // error case. we are in the spanning deserializer, but it has no bytes, yet
+// throw new IllegalStateException();
+ } else {
+ return DeserializationResult.PARTIAL_RECORD;
+ }
+ }
+
+ @Override
+ public void clear() {
+ this.nonSpanningWrapper.clear();
+ this.spanningWrapper.clear();
+ }
+
+ @Override
+ public boolean hasUnfinishedData() {
+ return this.nonSpanningWrapper.remaining() > 0 || this.spanningWrapper.getNumGatheredBytes() > 0;
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ private static final class NonSpanningWrapper implements DataInput {
+
+ private MemorySegment segment;
+
+ private int limit;
+
+ private int position;
+
+ private byte[] utfByteBuffer; // reusable byte buffer for utf-8 decoding
+ private char[] utfCharBuffer; // reusable char buffer for utf-8 decoding
+
+ int remaining() {
+ return this.limit - this.position;
+ }
+
+ void clear() {
+ this.segment = null;
+ this.limit = 0;
+ this.position = 0;
+ }
+
+ void initializeFromMemorySegment(MemorySegment seg, int position, int leftOverLimit) {
+ this.segment = seg;
+ this.position = position;
+ this.limit = leftOverLimit;
+ }
+
+ // -------------------------------------------------------------------------------------------------------------
+ // DataInput specific methods
+ // -------------------------------------------------------------------------------------------------------------
+
+ @Override
+ public final void readFully(byte[] b) throws IOException {
+ readFully(b, 0, b.length);
+ }
+
+ @Override
+ public final void readFully(byte[] b, int off, int len) throws IOException {
+ if (off < 0 || len < 0 || off + len > b.length)
+ throw new IndexOutOfBoundsException();
+
+ this.segment.get(this.position, b, off, len);
+ this.position += len;
+ }
+
+ @Override
+ public final boolean readBoolean() throws IOException {
+ return readByte() == 1;
+ }
+
+ @Override
+ public final byte readByte() throws IOException {
+ return this.segment.get(this.position++);
+ }
+
+ @Override
+ public final int readUnsignedByte() throws IOException {
+ return readByte() & 0xff;
+ }
+
+ @Override
+ public final short readShort() throws IOException {
+ final short v = this.segment.getShort(this.position);
+ this.position += 2;
+ return v;
+ }
+
+ @Override
+ public final int readUnsignedShort() throws IOException {
+ final int v = this.segment.getShort(this.position) & 0xffff;
+ this.position += 2;
+ return v;
+ }
+
+ @Override
+ public final char readChar() throws IOException {
+ final char v = this.segment.getChar(this.position);
+ this.position += 2;
+ return v;
+ }
+
+ @Override
+ public final int readInt() throws IOException {
+ final int v = this.segment.getIntBigEndian(this.position);
+ this.position += 4;
+ return v;
+ }
+
+ @Override
+ public final long readLong() throws IOException {
+ final long v = this.segment.getLongBigEndian(this.position);
+ this.position += 8;
+ return v;
+ }
+
+ @Override
+ public final float readFloat() throws IOException {
+ return Float.intBitsToFloat(readInt());
+ }
+
+ @Override
+ public final double readDouble() throws IOException {
+ return Double.longBitsToDouble(readLong());
+ }
+
+ @Override
+ public final String readLine() throws IOException {
+ final StringBuilder bld = new StringBuilder(32);
+
+ try {
+ int b;
+ while ((b = readUnsignedByte()) != '\n') {
+ if (b != '\r')
+ bld.append((char) b);
+ }
+ }
+ catch (EOFException eofex) {}
+
+ if (bld.length() == 0)
+ return null;
+
+ // trim a trailing carriage return
+ int len = bld.length();
+ if (len > 0 && bld.charAt(len - 1) == '\r') {
+ bld.setLength(len - 1);
+ }
+ return bld.toString();
+ }
+
+ @Override
+ public final String readUTF() throws IOException {
+ final int utflen = readUnsignedShort();
+
+ final byte[] bytearr;
+ final char[] chararr;
+
+ if (this.utfByteBuffer == null || this.utfByteBuffer.length < utflen) {
+ bytearr = new byte[utflen];
+ this.utfByteBuffer = bytearr;
+ } else {
+ bytearr = this.utfByteBuffer;
+ }
+ if (this.utfCharBuffer == null || this.utfCharBuffer.length < utflen) {
+ chararr = new char[utflen];
+ this.utfCharBuffer = chararr;
+ } else {
+ chararr = this.utfCharBuffer;
+ }
+
+ int c, char2, char3;
+ int count = 0;
+ int chararr_count = 0;
+
+ readFully(bytearr, 0, utflen);
+
+ while (count < utflen) {
+ c = (int) bytearr[count] & 0xff;
+ if (c > 127)
+ break;
+ count++;
+ chararr[chararr_count++] = (char) c;
+ }
+
+ while (count < utflen) {
+ c = (int) bytearr[count] & 0xff;
+ switch (c >> 4) {
+ case 0:
+ case 1:
+ case 2:
+ case 3:
+ case 4:
+ case 5:
+ case 6:
+ case 7:
+ count++;
+ chararr[chararr_count++] = (char) c;
+ break;
+ case 12:
+ case 13:
+ count += 2;
+ if (count > utflen)
+ throw new UTFDataFormatException("malformed input: partial character at end");
+ char2 = (int) bytearr[count - 1];
+ if ((char2 & 0xC0) != 0x80)
+ throw new UTFDataFormatException("malformed input around byte " + count);
+ chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
+ break;
+ case 14:
+ count += 3;
+ if (count > utflen)
+ throw new UTFDataFormatException("malformed input: partial character at end");
+ char2 = (int) bytearr[count - 2];
+ char3 = (int) bytearr[count - 1];
+ if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80))
+ throw new UTFDataFormatException("malformed input around byte " + (count - 1));
+ chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
+ break;
+ default:
+ throw new UTFDataFormatException("malformed input around byte " + count);
+ }
+ }
+ // The number of chars produced may be less than utflen
+ return new String(chararr, 0, chararr_count);
+ }
+
+ @Override
+ public final int skipBytes(int n) throws IOException {
+ if (n < 0)
+ throw new IllegalArgumentException();
+
+ int toSkip = Math.min(n, remaining());
+ this.position += toSkip;
+ return toSkip;
+ }
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ private static final class SpanningWrapper implements DataInput {
+
+ private final DataOutputSerializer serializationBuffer;
+
+ private final DataInputDeserializer serializationReadBuffer;
+
+ private final ByteBuffer lengthBuffer;
+
+ private int recordLength;
+
+ private MemorySegment leftOverData;
+
+ private int leftOverStart;
+
+ private int leftOverLimit;
+
+ private int recordLimit;
+
+ public SpanningWrapper() {
+ this.lengthBuffer = ByteBuffer.allocate(4);
+ this.lengthBuffer.order(ByteOrder.BIG_ENDIAN);
+
+ this.recordLength = -1;
+
+ this.serializationBuffer = new DataOutputSerializer(1024);
+ this.serializationReadBuffer = new DataInputDeserializer();
+ }
+
+ private void initializeWithPartialRecord(NonSpanningWrapper partial, int nextRecordLength) throws IOException {
+ // set the length and copy what is available to the buffer
+ this.recordLength = nextRecordLength;
+ this.recordLimit = partial.remaining();
+ partial.segment.get(this.serializationBuffer, partial.position, partial.remaining());
+ this.serializationReadBuffer.setBuffer(this.serializationBuffer.wrapAsByteBuffer());
+ }
+
+ private void initializeWithPartialLength(NonSpanningWrapper partial) throws IOException {
+ // copy what we have to the length buffer
+ partial.segment.get(partial.position, this.lengthBuffer, partial.remaining());
+ }
+
+ private void addNextChunkFromMemorySegment(MemorySegment segment, int numBytesInSegment) throws IOException {
+ int segmentPosition = 0;
+
+ // check where to go. if we have a partial length, we need to complete it first
+ if (this.lengthBuffer.position() > 0) {
+ int toPut = Math.min(this.lengthBuffer.remaining(), numBytesInSegment);
+ segment.get(0, this.lengthBuffer, toPut);
+
+ // did we complete the length?
+ if (this.lengthBuffer.hasRemaining()) {
+ return;
+ } else {
+ this.recordLength = this.lengthBuffer.getInt(0);
+ this.lengthBuffer.clear();
+ segmentPosition = toPut;
+ }
+ }
+
+ // copy as much as we need or can for this next spanning record
+ int needed = this.recordLength - this.recordLimit;
+ int available = numBytesInSegment - segmentPosition;
+ int toCopy = Math.min(needed, available);
+
+ segment.get(this.serializationBuffer, segmentPosition, toCopy);
+ this.recordLimit += toCopy;
+
+ if (toCopy < available) {
+ // there is more data in the segment
+ this.leftOverData = segment;
+ this.leftOverStart = segmentPosition + toCopy;
+ this.leftOverLimit = numBytesInSegment;
+ }
+
+ // update read view
+ this.serializationReadBuffer.setBuffer(this.serializationBuffer.wrapAsByteBuffer());
+ }
+
+ private void moveRemainderToNonSpanningDeserializer(NonSpanningWrapper deserializer) {
+ deserializer.clear();
+
+ if (leftOverData != null) {
+ deserializer.initializeFromMemorySegment(leftOverData, leftOverStart, leftOverLimit);
+ }
+ }
+
+ private boolean hasFullRecord() {
+ return this.recordLength >= 0 && this.recordLimit >= this.recordLength;
+ }
+
+ private int getNumGatheredBytes() {
+ return this.recordLimit + (this.recordLength >= 0 ? 4 : lengthBuffer.position()) + this.serializationBuffer.length();
+ }
+
+ public void clear() {
+ this.serializationBuffer.clear();
+
+ this.recordLength = -1;
+ this.lengthBuffer.clear();
+ this.leftOverData = null;
+ this.recordLimit = 0;
+ }
+
+ // -------------------------------------------------------------------------------------------------------------
+ // DataInput specific methods
+ // -------------------------------------------------------------------------------------------------------------
+
+ @Override
+ public void readFully(byte[] b) throws IOException {
+ this.serializationReadBuffer.readFully(b);
+ }
+
+ @Override
+ public void readFully(byte[] b, int off, int len) throws IOException {
+ this.serializationReadBuffer.readFully(b, off, len);
+ }
+
+ @Override
+ public int skipBytes(int n) throws IOException {
+ return this.serializationReadBuffer.skipBytes(n);
+ }
+
+ @Override
+ public boolean readBoolean() throws IOException {
+ return this.serializationReadBuffer.readBoolean();
+ }
+
+ @Override
+ public byte readByte() throws IOException {
+ return this.serializationReadBuffer.readByte();
+ }
+
+ @Override
+ public int readUnsignedByte() throws IOException {
+ return this.serializationReadBuffer.readUnsignedByte();
+ }
+
+ @Override
+ public short readShort() throws IOException {
+ return this.serializationReadBuffer.readShort();
+ }
+
+ @Override
+ public int readUnsignedShort() throws IOException {
+ return this.serializationReadBuffer.readUnsignedShort();
+ }
+
+ @Override
+ public char readChar() throws IOException {
+ return this.serializationReadBuffer.readChar();
+ }
+
+ @Override
+ public int readInt() throws IOException {
+ return this.serializationReadBuffer.readInt();
+ }
+
+ @Override
+ public long readLong() throws IOException {
+ return this.serializationReadBuffer.readLong();
+ }
+
+ @Override
+ public float readFloat() throws IOException {
+ return this.serializationReadBuffer.readFloat();
+ }
+
+ @Override
+ public double readDouble() throws IOException {
+ return this.serializationReadBuffer.readDouble();
+ }
+
+ @Override
+ public String readLine() throws IOException {
+ return this.serializationReadBuffer.readLine();
+ }
+
+ @Override
+ public String readUTF() throws IOException {
+ return this.serializationReadBuffer.readUTF();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java
new file mode 100644
index 0000000..e6479fe
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java
@@ -0,0 +1,307 @@
+/***********************************************************************************************************************
+ *
+ * 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.serialization;
+
+import eu.stratosphere.core.memory.MemoryUtils;
+
+import java.io.DataInput;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+/**
+ * A simple and efficient deserializer for the {@link java.io.DataInput} interface.
+ */
+public class DataInputDeserializer implements DataInput {
+
+ private byte[] buffer;
+
+ private int end;
+
+ private int position;
+
+ public DataInputDeserializer() {
+ }
+
+ public DataInputDeserializer(byte[] buffer, int start, int len) {
+ setBuffer(buffer, start, len);
+ }
+
+ public DataInputDeserializer(ByteBuffer buffer) {
+ setBuffer(buffer);
+ }
+
+ public void setBuffer(ByteBuffer buffer) {
+ if (buffer.hasArray()) {
+ this.buffer = buffer.array();
+ this.position = buffer.arrayOffset() + buffer.position();
+ this.end = this.position + buffer.remaining();
+ } else if (buffer.isDirect()) {
+ this.buffer = new byte[buffer.remaining()];
+ this.position = 0;
+ this.end = this.buffer.length;
+
+ buffer.get(this.buffer);
+ } else {
+ throw new IllegalArgumentException("The given buffer is neither an array-backed heap ByteBuffer, nor a direct ByteBuffer.");
+ }
+ }
+
+ public void setBuffer(byte[] buffer, int start, int len) {
+ if (buffer == null)
+ throw new NullPointerException();
+
+ if (start < 0 || len < 0 || start + len >= buffer.length)
+ throw new IllegalArgumentException();
+
+ this.buffer = buffer;
+ this.position = start;
+ this.end = start * len;
+ }
+
+ // ----------------------------------------------------------------------------------------
+ // Data Input
+ // ----------------------------------------------------------------------------------------
+
+ @Override
+ public boolean readBoolean() throws IOException {
+ if (this.position < this.end) {
+ return this.buffer[this.position++] != 0;
+ } else {
+ throw new EOFException();
+ }
+ }
+
+ @Override
+ public byte readByte() throws IOException {
+ if (this.position < this.end) {
+ return this.buffer[this.position++];
+ } else {
+ throw new EOFException();
+ }
+ }
+
+ @Override
+ public char readChar() throws IOException {
+ if (this.position < this.end - 1) {
+ return (char) (((this.buffer[this.position++] & 0xff) << 8) | ((this.buffer[this.position++] & 0xff) << 0));
+ } else {
+ throw new EOFException();
+ }
+ }
+
+ @Override
+ public double readDouble() throws IOException {
+ return Double.longBitsToDouble(readLong());
+ }
+
+ @Override
+ public float readFloat() throws IOException {
+ return Float.intBitsToFloat(readInt());
+ }
+
+ @Override
+ public void readFully(byte[] b) throws IOException {
+ readFully(b, 0, b.length);
+ }
+
+ @Override
+ public void readFully(byte[] b, int off, int len) throws IOException {
+ if (len >= 0) {
+ if (off <= b.length - len) {
+ if (this.position <= this.end - len) {
+ System.arraycopy(this.buffer, position, b, off, len);
+ position += len;
+ } else {
+ throw new EOFException();
+ }
+ } else {
+ throw new ArrayIndexOutOfBoundsException();
+ }
+ } else if (len < 0) {
+ throw new IllegalArgumentException("Length may not be negative.");
+ }
+ }
+
+ @Override
+ public int readInt() throws IOException {
+ if (this.position >= 0 && this.position < this.end - 3) {
+ @SuppressWarnings("restriction")
+ int value = UNSAFE.getInt(this.buffer, BASE_OFFSET + this.position);
+ if (LITTLE_ENDIAN) {
+ value = Integer.reverseBytes(value);
+ }
+
+ this.position += 4;
+ return value;
+ } else {
+ throw new EOFException();
+ }
+ }
+
+ @Override
+ public String readLine() throws IOException {
+ if (this.position < this.end) {
+ // read until a newline is found
+ StringBuilder bld = new StringBuilder();
+ char curr = (char) readUnsignedByte();
+ while (position < this.end && curr != '\n') {
+ bld.append(curr);
+ curr = (char) readUnsignedByte();
+ }
+ // trim a trailing carriage return
+ int len = bld.length();
+ if (len > 0 && bld.charAt(len - 1) == '\r') {
+ bld.setLength(len - 1);
+ }
+ String s = bld.toString();
+ bld.setLength(0);
+ return s;
+ } else {
+ return null;
+ }
+ }
+
+ @Override
+ public long readLong() throws IOException {
+ if (position >= 0 && position < this.end - 7) {
+ @SuppressWarnings("restriction")
+ long value = UNSAFE.getLong(this.buffer, BASE_OFFSET + this.position);
+ if (LITTLE_ENDIAN) {
+ value = Long.reverseBytes(value);
+ }
+ this.position += 8;
+ return value;
+ } else {
+ throw new EOFException();
+ }
+ }
+
+ @Override
+ public short readShort() throws IOException {
+ if (position >= 0 && position < this.end - 1) {
+ return (short) ((((this.buffer[position++]) & 0xff) << 8) | (((this.buffer[position++]) & 0xff) << 0));
+ } else {
+ throw new EOFException();
+ }
+ }
+
+ @Override
+ public String readUTF() throws IOException {
+ int utflen = readUnsignedShort();
+ byte[] bytearr = new byte[utflen];
+ char[] chararr = new char[utflen];
+
+ int c, char2, char3;
+ int count = 0;
+ int chararr_count = 0;
+
+ readFully(bytearr, 0, utflen);
+
+ while (count < utflen) {
+ c = (int) bytearr[count] & 0xff;
+ if (c > 127)
+ break;
+ count++;
+ chararr[chararr_count++] = (char) c;
+ }
+
+ while (count < utflen) {
+ c = (int) bytearr[count] & 0xff;
+ switch (c >> 4) {
+ case 0:
+ case 1:
+ case 2:
+ case 3:
+ case 4:
+ case 5:
+ case 6:
+ case 7:
+ /* 0xxxxxxx */
+ count++;
+ chararr[chararr_count++] = (char) c;
+ break;
+ case 12:
+ case 13:
+ /* 110x xxxx 10xx xxxx */
+ count += 2;
+ if (count > utflen)
+ throw new UTFDataFormatException("malformed input: partial character at end");
+ char2 = (int) bytearr[count - 1];
+ if ((char2 & 0xC0) != 0x80)
+ throw new UTFDataFormatException("malformed input around byte " + count);
+ chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
+ break;
+ case 14:
+ /* 1110 xxxx 10xx xxxx 10xx xxxx */
+ count += 3;
+ if (count > utflen)
+ throw new UTFDataFormatException("malformed input: partial character at end");
+ char2 = (int) bytearr[count - 2];
+ char3 = (int) bytearr[count - 1];
+ if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80))
+ throw new UTFDataFormatException("malformed input around byte " + (count - 1));
+ chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
+ break;
+ default:
+ /* 10xx xxxx, 1111 xxxx */
+ throw new UTFDataFormatException("malformed input around byte " + count);
+ }
+ }
+ // The number of chars produced may be less than utflen
+ return new String(chararr, 0, chararr_count);
+ }
+
+ @Override
+ public int readUnsignedByte() throws IOException {
+ if (this.position < this.end) {
+ return (this.buffer[this.position++] & 0xff);
+ } else {
+ throw new EOFException();
+ }
+ }
+
+ @Override
+ public int readUnsignedShort() throws IOException {
+ if (this.position < this.end - 1) {
+ return ((this.buffer[this.position++] & 0xff) << 8) | ((this.buffer[this.position++] & 0xff) << 0);
+ } else {
+ throw new EOFException();
+ }
+ }
+
+ @Override
+ public int skipBytes(int n) throws IOException {
+ if (this.position <= this.end - n) {
+ this.position += n;
+ return n;
+ } else {
+ n = this.end - this.position;
+ this.position = this.end;
+ return n;
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+
+ @SuppressWarnings("restriction")
+ private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+
+ private static final boolean LITTLE_ENDIAN = (MemoryUtils.NATIVE_BYTE_ORDER == ByteOrder.LITTLE_ENDIAN);
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java
new file mode 100644
index 0000000..b5171b9
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java
@@ -0,0 +1,259 @@
+/***********************************************************************************************************************
+ *
+ * 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.serialization;
+
+import eu.stratosphere.core.memory.MemoryUtils;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+/**
+ * A simple and efficient serializer for the {@link java.io.DataOutput} interface.
+ */
+public class DataOutputSerializer implements DataOutput {
+
+ private byte[] buffer;
+
+ private int position;
+
+ private ByteBuffer wrapper;
+
+ public DataOutputSerializer(int startSize) {
+ if (startSize < 1) {
+ throw new IllegalArgumentException();
+ }
+
+ this.buffer = new byte[startSize];
+ this.wrapper = ByteBuffer.wrap(buffer);
+ }
+
+ public ByteBuffer wrapAsByteBuffer() {
+ this.wrapper.position(0);
+ this.wrapper.limit(this.position);
+ return this.wrapper;
+ }
+
+ public void clear() {
+ this.position = 0;
+ }
+
+ public int length() {
+ return this.position;
+ }
+
+ @Override
+ public String toString() {
+ return String.format("[pos=%d cap=%d]", this.position, this.buffer.length);
+ }
+
+ // ----------------------------------------------------------------------------------------
+ // Data Output
+ // ----------------------------------------------------------------------------------------
+
+ @Override
+ public void write(int b) throws IOException {
+ if (this.position >= this.buffer.length) {
+ resize(1);
+ }
+ this.buffer[this.position++] = (byte) (b & 0xff);
+ }
+
+ @Override
+ public void write(byte[] b) throws IOException {
+ write(b, 0, b.length);
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ if (len < 0 || off > b.length - len) {
+ throw new ArrayIndexOutOfBoundsException();
+ }
+ if (this.position > this.buffer.length - len) {
+ resize(len);
+ }
+ System.arraycopy(b, off, this.buffer, this.position, len);
+ this.position += len;
+ }
+
+ @Override
+ public void writeBoolean(boolean v) throws IOException {
+ write(v ? 1 : 0);
+ }
+
+ @Override
+ public void writeByte(int v) throws IOException {
+ write(v);
+ }
+
+ @Override
+ public void writeBytes(String s) throws IOException {
+ final int sLen = s.length();
+ if (this.position >= this.buffer.length - sLen) {
+ resize(sLen);
+ }
+
+ for (int i = 0; i < sLen; i++) {
+ writeByte(s.charAt(i));
+ }
+ this.position += sLen;
+ }
+
+ @Override
+ public void writeChar(int v) throws IOException {
+ if (this.position >= this.buffer.length - 1) {
+ resize(2);
+ }
+ this.buffer[this.position++] = (byte) (v >> 8);
+ this.buffer[this.position++] = (byte) v;
+ }
+
+ @Override
+ public void writeChars(String s) throws IOException {
+ final int sLen = s.length();
+ if (this.position >= this.buffer.length - 2*sLen) {
+ resize(2*sLen);
+ }
+ for (int i = 0; i < sLen; i++) {
+ writeChar(s.charAt(i));
+ }
+ }
+
+ @Override
+ public void writeDouble(double v) throws IOException {
+ writeLong(Double.doubleToLongBits(v));
+ }
+
+ @Override
+ public void writeFloat(float v) throws IOException {
+ writeInt(Float.floatToIntBits(v));
+ }
+
+ @SuppressWarnings("restriction")
+ @Override
+ public void writeInt(int v) throws IOException {
+ if (this.position >= this.buffer.length - 3) {
+ resize(4);
+ }
+ if (LITTLE_ENDIAN) {
+ v = Integer.reverseBytes(v);
+ }
+ UNSAFE.putInt(this.buffer, BASE_OFFSET + this.position, v);
+ this.position += 4;
+ }
+
+ @SuppressWarnings("restriction")
+ @Override
+ public void writeLong(long v) throws IOException {
+ if (this.position >= this.buffer.length - 7) {
+ resize(8);
+ }
+ if (LITTLE_ENDIAN) {
+ v = Long.reverseBytes(v);
+ }
+ UNSAFE.putLong(this.buffer, BASE_OFFSET + this.position, v);
+ this.position += 8;
+ }
+
+ @Override
+ public void writeShort(int v) throws IOException {
+ if (this.position >= this.buffer.length - 1) {
+ resize(2);
+ }
+ this.buffer[this.position++] = (byte) ((v >>> 8) & 0xff);
+ this.buffer[this.position++] = (byte) ((v >>> 0) & 0xff);
+ }
+
+ @Override
+ public void writeUTF(String str) throws IOException {
+ int strlen = str.length();
+ int utflen = 0;
+ int c;
+
+ /* use charAt instead of copying String to char array */
+ for (int i = 0; i < strlen; i++) {
+ c = str.charAt(i);
+ if ((c >= 0x0001) && (c <= 0x007F)) {
+ utflen++;
+ } else if (c > 0x07FF) {
+ utflen += 3;
+ } else {
+ utflen += 2;
+ }
+ }
+
+ if (utflen > 65535)
+ throw new UTFDataFormatException("Encoded string is too long: " + utflen);
+
+ else if (this.position > this.buffer.length - utflen - 2) {
+ resize(utflen + 2);
+ }
+
+ byte[] bytearr = this.buffer;
+ int count = this.position;
+
+ bytearr[count++] = (byte) ((utflen >>> 8) & 0xFF);
+ bytearr[count++] = (byte) ((utflen >>> 0) & 0xFF);
+
+ int i = 0;
+ for (i = 0; i < strlen; i++) {
+ c = str.charAt(i);
+ if (!((c >= 0x0001) && (c <= 0x007F)))
+ break;
+ bytearr[count++] = (byte) c;
+ }
+
+ for (; i < strlen; i++) {
+ c = str.charAt(i);
+ if ((c >= 0x0001) && (c <= 0x007F)) {
+ bytearr[count++] = (byte) c;
+
+ } else if (c > 0x07FF) {
+ bytearr[count++] = (byte) (0xE0 | ((c >> 12) & 0x0F));
+ bytearr[count++] = (byte) (0x80 | ((c >> 6) & 0x3F));
+ bytearr[count++] = (byte) (0x80 | ((c >> 0) & 0x3F));
+ } else {
+ bytearr[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F));
+ bytearr[count++] = (byte) (0x80 | ((c >> 0) & 0x3F));
+ }
+ }
+
+ this.position = count;
+ }
+
+
+ private final void resize(int minCapacityAdd) throws IOException {
+ try {
+ final int newLen = Math.max(this.buffer.length * 2, this.buffer.length + minCapacityAdd);
+ final byte[] nb = new byte[newLen];
+ System.arraycopy(this.buffer, 0, nb, 0, this.position);
+ this.buffer = nb;
+ this.wrapper = ByteBuffer.wrap(this.buffer);
+ }
+ catch (NegativeArraySizeException nasex) {
+ throw new IOException("Serialization failed because the record length would exceed 2GB (max addressable array size in Java).");
+ }
+ }
+
+ @SuppressWarnings("restriction")
+ private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+
+ @SuppressWarnings("restriction")
+ private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+
+ private static final boolean LITTLE_ENDIAN = (MemoryUtils.NATIVE_BYTE_ORDER == ByteOrder.LITTLE_ENDIAN);
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordDeserializer.java
new file mode 100644
index 0000000..708c693
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordDeserializer.java
@@ -0,0 +1,56 @@
+/***********************************************************************************************************************
+ * 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.serialization;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.core.memory.MemorySegment;
+
+import java.io.IOException;
+
+/**
+ * Interface for turning sequences of memory segments into records.
+ */
+public interface RecordDeserializer<T extends IOReadableWritable> {
+
+ public static enum DeserializationResult {
+ PARTIAL_RECORD(false, true),
+ INTERMEDIATE_RECORD_FROM_BUFFER(true, false),
+ LAST_RECORD_FROM_BUFFER(true, true);
+
+ private final boolean isFullRecord;
+
+ private final boolean isBufferConsumed;
+
+ private DeserializationResult(boolean isFullRecord, boolean isBufferConsumed) {
+ this.isFullRecord = isFullRecord;
+ this.isBufferConsumed = isBufferConsumed;
+ }
+
+ public boolean isFullRecord () {
+ return this.isFullRecord;
+ }
+
+ public boolean isBufferConsumed() {
+ return this.isBufferConsumed;
+ }
+ }
+
+ DeserializationResult getNextRecord(T target) throws IOException;
+
+ void setNextMemorySegment(MemorySegment segment, int numBytes) throws IOException;
+
+ void clear();
+
+ boolean hasUnfinishedData();
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordSerializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordSerializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordSerializer.java
new file mode 100644
index 0000000..b540e27
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/RecordSerializer.java
@@ -0,0 +1,60 @@
+/***********************************************************************************************************************
+ *
+ * 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.serialization;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.runtime.io.Buffer;
+
+import java.io.IOException;
+
+/**
+ * Interface for turning records into sequences of memory segments.
+ */
+public interface RecordSerializer<T extends IOReadableWritable> {
+
+ public static enum SerializationResult {
+ PARTIAL_RECORD_MEMORY_SEGMENT_FULL(false, true),
+ FULL_RECORD_MEMORY_SEGMENT_FULL(true, true),
+ FULL_RECORD(true, false);
+
+ private final boolean isFullRecord;
+
+ private final boolean isFullBuffer;
+
+ private SerializationResult(boolean isFullRecord, boolean isFullBuffer) {
+ this.isFullRecord = isFullRecord;
+ this.isFullBuffer = isFullBuffer;
+ }
+
+ public boolean isFullRecord() {
+ return this.isFullRecord;
+ }
+
+ public boolean isFullBuffer() {
+ return this.isFullBuffer;
+ }
+ }
+
+ SerializationResult addRecord(T record) throws IOException;
+
+ SerializationResult setNextBuffer(Buffer buffer) throws IOException;
+
+ Buffer getCurrentBuffer();
+
+ void clear();
+
+ boolean hasData();
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java
new file mode 100644
index 0000000..443f8d8
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializer.java
@@ -0,0 +1,153 @@
+/***********************************************************************************************************************
+ * 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.serialization;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.Buffer;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+public class SpanningRecordSerializer<T extends IOReadableWritable> implements RecordSerializer<T> {
+
+ /** Flag to enable/disable checks, if buffer not set/full or pending serialization */
+ private static final boolean CHECKED = true;
+
+ /** Intermediate data serialization */
+ private final DataOutputSerializer serializationBuffer;
+
+ /** Intermediate buffer for data serialization */
+ private ByteBuffer dataBuffer;
+
+ /** Intermediate buffer for length serialization */
+ private final ByteBuffer lengthBuffer;
+
+ /** Current target {@link eu.stratosphere.runtime.io.Buffer} of the serializer */
+ private Buffer targetBuffer;
+
+ /** Position in current {@link MemorySegment} of target buffer */
+ private int position;
+
+ /** Limit of current {@link MemorySegment} of target buffer */
+ private int limit;
+
+ public SpanningRecordSerializer() {
+ this.serializationBuffer = new DataOutputSerializer(128);
+
+ this.lengthBuffer = ByteBuffer.allocate(4);
+ this.lengthBuffer.order(ByteOrder.BIG_ENDIAN);
+
+ // ensure initial state with hasRemaining false (for correct setNextBuffer logic)
+ this.dataBuffer = this.serializationBuffer.wrapAsByteBuffer();
+ this.lengthBuffer.position(4);
+ }
+
+ @Override
+ public SerializationResult addRecord(T record) throws IOException {
+ if (CHECKED) {
+ if (this.dataBuffer.hasRemaining()) {
+ throw new IllegalStateException("Pending serialization of previous record.");
+ }
+ }
+
+ this.serializationBuffer.clear();
+ this.lengthBuffer.clear();
+
+ // write data and length
+ record.write(this.serializationBuffer);
+ this.lengthBuffer.putInt(0, this.serializationBuffer.length());
+
+ this.dataBuffer = this.serializationBuffer.wrapAsByteBuffer();
+
+ // Copy from intermediate buffers to current target memory segment
+ copyToTargetBufferFrom(this.lengthBuffer);
+ copyToTargetBufferFrom(this.dataBuffer);
+
+ return getSerializationResult();
+ }
+
+ @Override
+ public SerializationResult setNextBuffer(Buffer buffer) throws IOException {
+ this.targetBuffer = buffer;
+ this.position = 0;
+ this.limit = buffer.size();
+
+ if (this.lengthBuffer.hasRemaining()) {
+ copyToTargetBufferFrom(this.lengthBuffer);
+ }
+
+ if (this.dataBuffer.hasRemaining()) {
+ copyToTargetBufferFrom(this.dataBuffer);
+ }
+
+ return getSerializationResult();
+ }
+
+ /**
+ * Copies as many bytes as possible from the given {@link ByteBuffer} to the {@link MemorySegment} of the target
+ * {@link Buffer} and advances the current position by the number of written bytes.
+ *
+ * @param source the {@link ByteBuffer} to copy data from
+ */
+ private void copyToTargetBufferFrom(ByteBuffer source) {
+ if (this.targetBuffer == null)
+ return;
+
+ int needed = source.remaining();
+ int available = this.limit - this.position;
+ int toCopy = Math.min(needed, available);
+
+ this.targetBuffer.getMemorySegment().put(this.position, source, toCopy);
+
+ this.position += toCopy;
+ }
+
+ private SerializationResult getSerializationResult() {
+ if (!this.dataBuffer.hasRemaining() && !this.lengthBuffer.hasRemaining()) {
+ return (this.position < this.limit)
+ ? SerializationResult.FULL_RECORD
+ : SerializationResult.FULL_RECORD_MEMORY_SEGMENT_FULL;
+ }
+
+ return SerializationResult.PARTIAL_RECORD_MEMORY_SEGMENT_FULL;
+ }
+
+ @Override
+ public Buffer getCurrentBuffer() {
+ if (targetBuffer == null)
+ return null;
+
+ this.targetBuffer.limitSize(this.position);
+ return this.targetBuffer;
+ }
+
+ @Override
+ public void clear() {
+ this.targetBuffer = null;
+ this.position = 0;
+ this.limit = 0;
+
+ // ensure clear state with hasRemaining false (for correct setNextBuffer logic)
+ this.dataBuffer.position(this.dataBuffer.limit());
+ this.lengthBuffer.position(4);
+ }
+
+ @Override
+ public boolean hasData() {
+ // either data in current target buffer or intermediate buffers
+ return this.position > 0 || (this.lengthBuffer.hasRemaining() || this.dataBuffer.hasRemaining());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/AbstractIDTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/AbstractIDTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/AbstractIDTest.java
new file mode 100644
index 0000000..65a7b19
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/AbstractIDTest.java
@@ -0,0 +1,62 @@
+/***********************************************************************************************************************
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.nephele.util.CommonTestUtils;
+
+/**
+ * This class contains tests for the {@link eu.stratosphere.nephele.AbstractID} class.
+ *
+ */
+public class AbstractIDTest {
+
+ /**
+ * Tests the setID method of an abstract ID.
+ */
+ @Test
+ public void testSetID() {
+
+ final ChannelID id1 = new ChannelID();
+ final ChannelID id2 = new ChannelID();
+ id1.setID(id2);
+
+ assertEquals(id1.hashCode(), id2.hashCode());
+ assertEquals(id1, id2);
+ }
+
+ /**
+ * Tests the serialization/deserialization of an abstract ID.
+ */
+ @Test
+ public void testSerialization() {
+
+ final ChannelID origID = new ChannelID();
+ try {
+ final ChannelID copyID = (ChannelID) CommonTestUtils.createCopy(origID);
+
+ assertEquals(origID.hashCode(), copyID.hashCode());
+ assertEquals(origID, copyID);
+
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptorTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptorTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptorTest.java
index 7cf83d7..b3ff279 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptorTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptorTest.java
@@ -21,7 +21,7 @@ import java.io.IOException;
import org.junit.Test;
-import eu.stratosphere.nephele.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
import eu.stratosphere.nephele.util.ServerTestUtils;
import eu.stratosphere.util.StringUtils;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptorTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptorTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptorTest.java
index 66ccad8..bc7034f 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptorTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptorTest.java
@@ -23,9 +23,9 @@ import java.util.List;
import org.junit.Test;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
+import eu.stratosphere.runtime.io.gates.GateID;
+import eu.stratosphere.runtime.io.channels.ChannelID;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.nephele.util.ServerTestUtils;
import eu.stratosphere.util.StringUtils;
@@ -42,7 +42,7 @@ public class GateDeploymentDescriptorTest {
public void testConstructorWithValidArguments() {
final GateID gateID = new GateID();
- final ChannelType channelType = ChannelType.INMEMORY;
+ final ChannelType channelType = ChannelType.IN_MEMORY;
final List<ChannelDeploymentDescriptor> channels = new ArrayList<ChannelDeploymentDescriptor>(0);
final GateDeploymentDescriptor gdd = new GateDeploymentDescriptor(gateID, channelType, channels);
@@ -59,7 +59,7 @@ public class GateDeploymentDescriptorTest {
public void testConstructorWithInvalidArguments() {
final GateID gateID = new GateID();
- final ChannelType channelType = ChannelType.INMEMORY;
+ final ChannelType channelType = ChannelType.IN_MEMORY;
final List<ChannelDeploymentDescriptor> channels = new ArrayList<ChannelDeploymentDescriptor>(0);
boolean firstExceptionCaught = false;
@@ -105,7 +105,7 @@ public class GateDeploymentDescriptorTest {
public void testSerialization() {
final GateID gateID = new GateID();
- final ChannelType channelType = ChannelType.INMEMORY;
+ final ChannelType channelType = ChannelType.IN_MEMORY;
final List<ChannelDeploymentDescriptor> channels = new ArrayList<ChannelDeploymentDescriptor>(0);
final ChannelDeploymentDescriptor cdd = new ChannelDeploymentDescriptor(new ChannelID(), new ChannelID());
channels.add(cdd);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/TaskDeploymentDescriptorTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/TaskDeploymentDescriptorTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/TaskDeploymentDescriptorTest.java
index e2581f8..7000667 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/TaskDeploymentDescriptorTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/deployment/TaskDeploymentDescriptorTest.java
@@ -24,7 +24,7 @@ import org.junit.Test;
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager;
import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
-import eu.stratosphere.nephele.io.library.FileLineReader;
+import eu.stratosphere.nephele.util.FileLineReader;
import eu.stratosphere.nephele.jobgraph.JobID;
import eu.stratosphere.nephele.template.AbstractInvokable;
import eu.stratosphere.nephele.util.SerializableArrayList;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
index 25e538d..5ff5f1c 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ExecutionGraphTest.java
@@ -45,10 +45,10 @@ import eu.stratosphere.nephele.instance.InstanceRequestMap;
import eu.stratosphere.nephele.instance.InstanceType;
import eu.stratosphere.nephele.instance.InstanceTypeDescription;
import eu.stratosphere.nephele.instance.InstanceTypeFactory;
-import eu.stratosphere.nephele.io.DistributionPattern;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.library.FileLineReader;
-import eu.stratosphere.nephele.io.library.FileLineWriter;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
+import eu.stratosphere.nephele.util.FileLineReader;
+import eu.stratosphere.nephele.util.FileLineWriter;
import eu.stratosphere.nephele.jobgraph.JobFileInputVertex;
import eu.stratosphere.nephele.jobgraph.JobFileOutputVertex;
import eu.stratosphere.nephele.jobgraph.JobGraph;
@@ -448,7 +448,7 @@ public class ExecutionGraphTest {
* input1 -> task1 -> output1
* no subtasks defined
* input1 is default, task1 is m1.large, output1 is m1.xlarge
- * all channels are INMEMORY
+ * all channels are IN_MEMORY
*/
@Test
public void testConvertJobGraphToExecutionGraph2() {
@@ -478,8 +478,8 @@ public class ExecutionGraphTest {
o1.setFilePath(new Path(new File(ServerTestUtils.getRandomFilename()).toURI()));
// connect vertices
- i1.connectTo(t1, ChannelType.INMEMORY);
- t1.connectTo(o1, ChannelType.INMEMORY);
+ i1.connectTo(t1, ChannelType.IN_MEMORY);
+ t1.connectTo(o1, ChannelType.IN_MEMORY);
LibraryCacheManager.register(jobID, new String[0]);
@@ -865,11 +865,11 @@ public class ExecutionGraphTest {
o1.setVertexToShareInstancesWith(o2);
// connect vertices
- i1.connectTo(t1, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
- i2.connectTo(t2, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ i1.connectTo(t1, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
+ i2.connectTo(t2, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
t1.connectTo(t3, ChannelType.NETWORK);
t2.connectTo(t3, ChannelType.NETWORK);
- t3.connectTo(t4, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ t3.connectTo(t4, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
t4.connectTo(o1, ChannelType.NETWORK);
t4.connectTo(o2, ChannelType.NETWORK);
@@ -973,11 +973,11 @@ public class ExecutionGraphTest {
output.setNumberOfSubtasks(degreeOfParallelism);
// connect vertices
- input.connectTo(cross, ChannelType.INMEMORY, 0, 0,
+ input.connectTo(cross, ChannelType.IN_MEMORY, 0, 0,
DistributionPattern.POINTWISE);
input.connectTo(cross, ChannelType.NETWORK, 1, 1,
DistributionPattern.BIPARTITE);
- cross.connectTo(output, ChannelType.INMEMORY, 0, 0,
+ cross.connectTo(output, ChannelType.IN_MEMORY, 0, 0,
DistributionPattern.POINTWISE);
LibraryCacheManager.register(jobID, new String[0]);
@@ -1113,13 +1113,13 @@ public class ExecutionGraphTest {
output1.setNumberOfSubtasks(degreeOfParallelism);
// connect vertices
- input1.connectTo(forward1, ChannelType.INMEMORY,
+ input1.connectTo(forward1, ChannelType.IN_MEMORY,
DistributionPattern.POINTWISE);
- forward1.connectTo(forward2, ChannelType.INMEMORY,
+ forward1.connectTo(forward2, ChannelType.IN_MEMORY,
DistributionPattern.POINTWISE);
forward2.connectTo(forward3, ChannelType.NETWORK,
DistributionPattern.POINTWISE);
- forward3.connectTo(output1, ChannelType.INMEMORY);
+ forward3.connectTo(output1, ChannelType.IN_MEMORY);
// setup instance sharing
input1.setVertexToShareInstancesWith(forward1);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input1Output.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input1Output.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input1Output.java
index 21341fa..0a2f52b 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input1Output.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input1Output.java
@@ -14,8 +14,8 @@
package eu.stratosphere.nephele.executiongraph;
import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.nephele.template.AbstractTask;
public class ForwardTask1Input1Output extends AbstractTask {
@@ -26,18 +26,21 @@ public class ForwardTask1Input1Output extends AbstractTask {
@Override
public void invoke() throws Exception {
+ this.output.initializeSerializers();
while (this.input.hasNext()) {
StringRecord s = input.next();
this.output.emit(s);
}
+
+ this.output.flush();
}
@Override
public void registerInputOutput() {
this.input = new RecordReader<StringRecord>(this, StringRecord.class);
- this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
+ this.output = new RecordWriter<StringRecord>(this);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input2Outputs.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input2Outputs.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input2Outputs.java
index 6582ea7..5a5c325 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input2Outputs.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask1Input2Outputs.java
@@ -14,8 +14,8 @@
package eu.stratosphere.nephele.executiongraph;
import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.nephele.template.AbstractTask;
public class ForwardTask1Input2Outputs extends AbstractTask {
@@ -29,18 +29,24 @@ public class ForwardTask1Input2Outputs extends AbstractTask {
@Override
public void invoke() throws Exception {
+ this.output1.initializeSerializers();
+ this.output2.initializeSerializers();
+
while (this.input.hasNext()) {
StringRecord s = input.next();
this.output1.emit(s);
this.output2.emit(s);
}
+
+ this.output1.flush();
+ this.output2.flush();
}
@Override
public void registerInputOutput() {
this.input = new RecordReader<StringRecord>(this, StringRecord.class);
- this.output1 = new RecordWriter<StringRecord>(this, StringRecord.class);
- this.output2 = new RecordWriter<StringRecord>(this, StringRecord.class);
+ this.output1 = new RecordWriter<StringRecord>(this);
+ this.output2 = new RecordWriter<StringRecord>(this);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask2Inputs1Output.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask2Inputs1Output.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask2Inputs1Output.java
index 7e8af0b..c87d093 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask2Inputs1Output.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/ForwardTask2Inputs1Output.java
@@ -14,8 +14,8 @@
package eu.stratosphere.nephele.executiongraph;
import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.nephele.template.AbstractTask;
public class ForwardTask2Inputs1Output extends AbstractTask {
@@ -28,6 +28,7 @@ public class ForwardTask2Inputs1Output extends AbstractTask {
@Override
public void invoke() throws Exception {
+ this.output.initializeSerializers();
while (this.input1.hasNext()) {
@@ -44,12 +45,14 @@ public class ForwardTask2Inputs1Output extends AbstractTask {
e.printStackTrace();
}
}
+
+ this.output.flush();
}
@Override
public void registerInputOutput() {
this.input1 = new RecordReader<StringRecord>(this, StringRecord.class);
this.input2 = new RecordReader<StringRecord>(this, StringRecord.class);
- this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
+ this.output = new RecordWriter<StringRecord>(this);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossForwardTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossForwardTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossForwardTask.java
index c375fd4..05f181c 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossForwardTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossForwardTask.java
@@ -15,8 +15,8 @@ package eu.stratosphere.nephele.executiongraph;
import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.nephele.template.AbstractTask;
/**
@@ -31,7 +31,7 @@ public class SelfCrossForwardTask extends AbstractTask {
new RecordReader<StringRecord>(this, StringRecord.class);
new RecordReader<StringRecord>(this, StringRecord.class);
- new RecordWriter<StringRecord>(this, StringRecord.class);
+ new RecordWriter<StringRecord>(this);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossInputTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossInputTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossInputTask.java
index a2af3a9..1ce23e6 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossInputTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/executiongraph/SelfCrossInputTask.java
@@ -14,7 +14,7 @@
package eu.stratosphere.nephele.executiongraph;
import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.nephele.template.AbstractFileInputTask;
/**
@@ -27,8 +27,8 @@ public class SelfCrossInputTask extends AbstractFileInputTask {
@Override
public void registerInputOutput() {
- new RecordWriter<StringRecord>(this, StringRecord.class);
- new RecordWriter<StringRecord>(this, StringRecord.class);
+ new RecordWriter<StringRecord>(this);
+ new RecordWriter<StringRecord>(this);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/LineReaderTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/LineReaderTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/LineReaderTest.java
deleted file mode 100644
index 00dd645..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/LineReaderTest.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/***********************************************************************************************************************
- * 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.fs;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.PrintWriter;
-
-import org.junit.Test;
-
-import eu.stratosphere.core.fs.FSDataInputStream;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.core.fs.local.LocalFileSystem;
-import eu.stratosphere.nephele.util.CommonTestUtils;
-import eu.stratosphere.runtime.fs.LineReader;
-
-/**
- * This class tests the functionality of the LineReader class using a local filesystem.
- *
- */
-
-public class LineReaderTest {
-
- /**
- * This test tests the LineReader. So far only under usual conditions.
- */
- @Test
- public void testLineReader() {
- final File testfile = new File(CommonTestUtils.getTempDir() + File.separator
- + CommonTestUtils.getRandomFilename());
- final Path pathtotestfile = new Path(testfile.toURI().getPath());
-
- try {
- PrintWriter pw = new PrintWriter(testfile, "UTF8");
-
- for (int i = 0; i < 100; i++) {
- pw.append("line\n");
- }
- pw.close();
-
- LocalFileSystem lfs = new LocalFileSystem();
- FSDataInputStream fis = lfs.open(pathtotestfile);
-
- // first, we test under "usual" conditions
- final LineReader lr = new LineReader(fis, 0, testfile.length(), 256);
-
- byte[] buffer;
- int linecount = 0;
- while ((buffer = lr.readLine()) != null) {
- assertEquals(new String(buffer, "UTF8"), "line");
- linecount++;
- }
- assertEquals(linecount, 100);
-
- // the linereader can not handle situations with larger length than the total file...
-
- } catch (Exception e) {
- fail(e.toString());
- e.printStackTrace();
- } finally {
- testfile.delete();
- }
-
- }
-
-}
[29/30] git commit: Remove @Test from NettyConnectionManagerTest and
run tests via main method
Posted by rm...@apache.org.
Remove @Test from NettyConnectionManagerTest and run tests via main method
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/b0f8ba06
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/b0f8ba06
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/b0f8ba06
Branch: refs/heads/master
Commit: b0f8ba06953f978516044db220a89cacbcbb2d5c
Parents: 6a40d41
Author: uce <u....@fu-berlin.de>
Authored: Tue Jun 3 15:14:22 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Sat Jun 7 09:41:22 2014 +0200
----------------------------------------------------------------------
.../netty/NettyConnectionManagerTest.java | 26 +++++++++++++++++---
1 file changed, 22 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b0f8ba06/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java
index c424a1f..c380431 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java
@@ -19,7 +19,6 @@ 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;
@@ -69,9 +68,9 @@ public class NettyConnectionManagerTest {
}
@Parameterized.Parameters
- public static Collection configure() {
+ public static Collection<Integer[]> configure() {
return Arrays.asList(
- new Object[][]{
+ new Integer[][]{
{64, 4096, 1, 1, 1},
{128, 2048, 1, 1, 1},
{256, 1024, 1, 1, 1},
@@ -88,7 +87,6 @@ public class NettyConnectionManagerTest {
);
}
- @Test
public void testEnqueueRaceAndDeadlockFreeMultipleChannels() throws Exception {
final InetAddress localhost = InetAddress.getLocalHost();
final CountDownLatch latch = new CountDownLatch(this.numSubtasks);
@@ -193,4 +191,24 @@ public class NettyConnectionManagerTest {
}
}
}
+
+ private void runAllTests() throws Exception {
+ System.out.println(String.format("Running tests with config: %d sub tasks, %d envelopes to send per subtasks, "
+ + "%d num channels, %d num in threads, %d num out threads.",
+ this.numSubtasks, this.numToSendPerSubtask, this.numChannels, this.numInThreads, this.numOutThreads));
+
+ testEnqueueRaceAndDeadlockFreeMultipleChannels();
+
+ System.out.println("Done.");
+ }
+
+ public static void main(String[] args) throws Exception {
+ Collection<Integer[]> configs = configure();
+
+ for (Integer[] params : configs) {
+
+ NettyConnectionManagerTest test = new NettyConnectionManagerTest(params[0], params[1], params[2], params[3], params[4]);
+ test.runAllTests();
+ }
+ }
}
[11/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/s3/S3FileSystemTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/s3/S3FileSystemTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/s3/S3FileSystemTest.java
deleted file mode 100644
index 784178d..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/fs/s3/S3FileSystemTest.java
+++ /dev/null
@@ -1,461 +0,0 @@
-/***********************************************************************************************************************
- * 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.fs.s3;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.URI;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.configuration.GlobalConfiguration;
-import eu.stratosphere.core.fs.BlockLocation;
-import eu.stratosphere.core.fs.FSDataInputStream;
-import eu.stratosphere.core.fs.FSDataOutputStream;
-import eu.stratosphere.core.fs.FileStatus;
-import eu.stratosphere.core.fs.FileSystem;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.runtime.fs.s3.S3FileSystem;
-
-/**
- * This test checks the S3 implementation of the {@link FileSystem} interface.
- *
- */
-public class S3FileSystemTest {
-
- /**
- * The length of the bucket/object names used in this test.
- */
- private static final int NAME_LENGTH = 32;
-
- /**
- * The alphabet to generate the random bucket/object names from.
- */
- private static final char[] ALPHABET = { 'a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm', 'n', 'o',
- 'p', 'q', 'r', 's', 't', 'u', 'v', 'w', 'x', 'y', 'z', '0', '1', '2', '3', '4', '5', '6', '7', '8', '9' };
-
- /**
- * The size of the byte buffer used during the tests in bytes.
- */
- private static final int TEST_BUFFER_SIZE = 128;
-
- /**
- * The size of the small test file in bytes.
- */
- private static final int SMALL_FILE_SIZE = 512;
-
- /**
- * The size of the large test file in bytes.
- */
- private static final int LARGE_FILE_SIZE = 1024 * 1024 * 12; // 12 MB
-
- /**
- * The modulus to be used when generating the test data. Must not be larger than 128.
- */
- private static final int MODULUS = 128;
-
- private static final String S3_BASE_URI = "s3:///";
-
- /**
- * Tries to read the AWS access key and the AWS secret key from the environments variables. If accessing these keys
- * fails, all tests will be skipped and marked as successful.
- */
- @Before
- public void initKeys() {
- final String accessKey = System.getenv("AK");
- final String secretKey = System.getenv("SK");
-
- if (accessKey != null || secretKey != null) {
- Configuration conf = new Configuration();
- if (accessKey != null) {
- conf.setString(S3FileSystem.S3_ACCESS_KEY_KEY, accessKey);
- }
- if (secretKey != null) {
- conf.setString(S3FileSystem.S3_SECRET_KEY_KEY, secretKey);
- }
- GlobalConfiguration.includeConfiguration(conf);
- }
- }
-
- /**
- * This test creates and deletes a bucket inside S3 and checks it is correctly displayed inside the directory
- * listing.
- */
- @Test
- public void createAndDeleteBucketTest() {
-
- if (!testActivated()) {
- return;
- }
-
- final String bucketName = getRandomName();
- final Path bucketPath = new Path(S3_BASE_URI + bucketName + Path.SEPARATOR);
-
- try {
-
- final FileSystem fs = bucketPath.getFileSystem();
-
- // Create directory
- fs.mkdirs(bucketPath);
-
- // Check if directory is correctly displayed in file system hierarchy
- final FileStatus[] content = fs.listStatus(new Path(S3_BASE_URI));
- boolean entryFound = false;
- for (final FileStatus entry : content) {
- if (bucketPath.equals(entry.getPath())) {
- entryFound = true;
- break;
- }
- }
-
- if (!entryFound) {
- fail("Cannot find entry " + bucketName + " in directory " + S3_BASE_URI);
- }
-
- // Check the concrete directory file status
- try {
- final FileStatus directoryFileStatus = fs.getFileStatus(bucketPath);
- assertTrue(directoryFileStatus.isDir());
- assertEquals(0L, directoryFileStatus.getAccessTime());
- assertTrue(directoryFileStatus.getModificationTime() > 0L);
-
- } catch (FileNotFoundException e) {
- fail(e.getMessage());
- }
-
- // Delete the bucket
- fs.delete(bucketPath, true);
-
- // Make sure the bucket no longer exists
- try {
- fs.getFileStatus(bucketPath);
- fail("Expected FileNotFoundException for " + bucketPath.toUri());
- } catch (FileNotFoundException e) {
- // This is an expected exception
- }
-
- } catch (IOException ioe) {
- fail(ioe.getMessage());
- }
- }
-
- /**
- * Creates and reads the a larger test file in S3. The test file is generated according to a specific pattern.
- * During the read phase the incoming data stream is also checked against this pattern.
- */
- @Test
- public void createAndReadLargeFileTest() {
-
- try {
- createAndReadFileTest(LARGE_FILE_SIZE);
- } catch (IOException ioe) {
- fail(ioe.getMessage());
- }
- }
-
- /**
- * Creates and reads the a small test file in S3. The test file is generated according to a specific pattern.
- * During the read phase the incoming data stream is also checked against this pattern.
- */
- @Test
- public void createAndReadSmallFileTest() {
-
- try {
- createAndReadFileTest(SMALL_FILE_SIZE);
- } catch (IOException ioe) {
- fail(ioe.getMessage());
- }
- }
-
- /**
- * The tests checks the mapping of the file system directory structure to the underlying bucket/object model of
- * Amazon S3.
- */
- @Test
- public void multiLevelDirectoryTest() {
-
- if (!testActivated()) {
- return;
- }
-
- final String dirName = getRandomName();
- final String subdirName = getRandomName();
- final String subsubdirName = getRandomName();
- final String fileName = getRandomName();
- final Path dir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR);
- final Path subdir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + subdirName + Path.SEPARATOR);
- final Path subsubdir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + subdirName + Path.SEPARATOR
- + subsubdirName + Path.SEPARATOR);
- final Path file = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + subdirName + Path.SEPARATOR + fileName);
-
- try {
-
- final FileSystem fs = dir.getFileSystem();
-
- fs.mkdirs(subsubdir);
-
- final OutputStream os = fs.create(file, true);
- generateTestData(os, SMALL_FILE_SIZE);
- os.close();
-
- // On this directory levels there should only be one subdirectory
- FileStatus[] list = fs.listStatus(dir);
- int numberOfDirs = 0;
- int numberOfFiles = 0;
- for (final FileStatus entry : list) {
-
- if (entry.isDir()) {
- ++numberOfDirs;
- assertEquals(subdir, entry.getPath());
- } else {
- fail(entry.getPath() + " is a file which must not appear on this directory level");
- }
- }
-
- assertEquals(1, numberOfDirs);
- assertEquals(0, numberOfFiles);
-
- list = fs.listStatus(subdir);
- numberOfDirs = 0;
-
- for (final FileStatus entry : list) {
- if (entry.isDir()) {
- assertEquals(subsubdir, entry.getPath());
- ++numberOfDirs;
- } else {
- assertEquals(file, entry.getPath());
- ++numberOfFiles;
- }
- }
-
- assertEquals(1, numberOfDirs);
- assertEquals(1, numberOfFiles);
-
- fs.delete(dir, true);
-
- } catch (IOException ioe) {
- fail(ioe.getMessage());
- }
- }
-
- /**
- * This test checks the S3 implementation of the file system method to retrieve the block locations of a file.
- */
- @Test
- public void blockLocationTest() {
-
- if (!testActivated()) {
- return;
- }
-
- final String dirName = getRandomName();
- final String fileName = getRandomName();
- final Path dir = new Path(S3_BASE_URI + dirName + Path.SEPARATOR);
- final Path file = new Path(S3_BASE_URI + dirName + Path.SEPARATOR + fileName);
-
- try {
-
- final FileSystem fs = dir.getFileSystem();
-
- fs.mkdirs(dir);
-
- final OutputStream os = fs.create(file, true);
- generateTestData(os, SMALL_FILE_SIZE);
- os.close();
-
- final FileStatus fileStatus = fs.getFileStatus(file);
- assertNotNull(fileStatus);
-
- BlockLocation[] blockLocations = fs.getFileBlockLocations(fileStatus, 0, SMALL_FILE_SIZE + 1);
- assertNull(blockLocations);
-
- blockLocations = fs.getFileBlockLocations(fileStatus, 0, SMALL_FILE_SIZE);
- assertEquals(1, blockLocations.length);
-
- final BlockLocation bl = blockLocations[0];
- assertNotNull(bl.getHosts());
- assertEquals(1, bl.getHosts().length);
- assertEquals(SMALL_FILE_SIZE, bl.getLength());
- assertEquals(0, bl.getOffset());
- final URI s3Uri = fs.getUri();
- assertNotNull(s3Uri);
- assertEquals(s3Uri.getHost(), bl.getHosts()[0]);
-
- fs.delete(dir, true);
-
- } catch (IOException ioe) {
- fail(ioe.getMessage());
- }
- }
-
- /**
- * Creates and reads a file with the given size in S3. The test file is generated according to a specific pattern.
- * During the read phase the incoming data stream is also checked against this pattern.
- *
- * @param fileSize
- * the size of the file to be generated in bytes
- * @throws IOException
- * thrown if an I/O error occurs while writing or reading the test file
- */
- private void createAndReadFileTest(final int fileSize) throws IOException {
-
- if (!testActivated()) {
- return;
- }
-
- final String bucketName = getRandomName();
- final String objectName = getRandomName();
- final Path bucketPath = new Path(S3_BASE_URI + bucketName + Path.SEPARATOR);
- final Path objectPath = new Path(S3_BASE_URI + bucketName + Path.SEPARATOR + objectName);
-
- FileSystem fs = bucketPath.getFileSystem();
-
- // Create test bucket
- fs.mkdirs(bucketPath);
-
- // Write test file to S3
- final FSDataOutputStream outputStream = fs.create(objectPath, false);
- generateTestData(outputStream, fileSize);
- outputStream.close();
-
- // Now read the same file back from S3
- final FSDataInputStream inputStream = fs.open(objectPath);
- testReceivedData(inputStream, fileSize);
- inputStream.close();
-
- // Delete test bucket
- fs.delete(bucketPath, true);
- }
-
- /**
- * Receives test data from the given input stream and checks the size of the data as well as the pattern inside the
- * received data.
- *
- * @param inputStream
- * the input stream to read the test data from
- * @param expectedSize
- * the expected size of the data to be read from the input stream in bytes
- * @throws IOException
- * thrown if an error occurs while reading the data
- */
- private void testReceivedData(final InputStream inputStream, final int expectedSize) throws IOException {
-
- final byte[] testBuffer = new byte[TEST_BUFFER_SIZE];
-
- int totalBytesRead = 0;
- int nextExpectedNumber = 0;
- while (true) {
-
- final int bytesRead = inputStream.read(testBuffer);
- if (bytesRead < 0) {
- break;
- }
-
- totalBytesRead += bytesRead;
-
- for (int i = 0; i < bytesRead; ++i) {
- if (testBuffer[i] != nextExpectedNumber) {
- throw new IOException("Read number " + testBuffer[i] + " but expected " + nextExpectedNumber);
- }
-
- ++nextExpectedNumber;
-
- if (nextExpectedNumber == MODULUS) {
- nextExpectedNumber = 0;
- }
- }
- }
-
- if (totalBytesRead != expectedSize) {
- throw new IOException("Expected to read " + expectedSize + " bytes but only received " + totalBytesRead);
- }
- }
-
- /**
- * Generates test data of the given size according to some specific pattern and writes it to the provided output
- * stream.
- *
- * @param outputStream
- * the output stream to write the data to
- * @param size
- * the size of the test data to be generated in bytes
- * @throws IOException
- * thrown if an error occurs while writing the data
- */
- private void generateTestData(final OutputStream outputStream, final int size) throws IOException {
-
- final byte[] testBuffer = new byte[TEST_BUFFER_SIZE];
- for (int i = 0; i < testBuffer.length; ++i) {
- testBuffer[i] = (byte) (i % MODULUS);
- }
-
- int bytesWritten = 0;
- while (bytesWritten < size) {
-
- final int diff = size - bytesWritten;
- if (diff < testBuffer.length) {
- outputStream.write(testBuffer, 0, diff);
- bytesWritten += diff;
- } else {
- outputStream.write(testBuffer);
- bytesWritten += testBuffer.length;
- }
- }
- }
-
- /**
- * Generates a random name.
- *
- * @return a random name
- */
- private String getRandomName() {
-
- final StringBuilder stringBuilder = new StringBuilder();
- for (int i = 0; i < NAME_LENGTH; ++i) {
- final char c = ALPHABET[(int) (Math.random() * (double) ALPHABET.length)];
- stringBuilder.append(c);
- }
-
- return stringBuilder.toString();
- }
-
- /**
- * Checks whether the AWS access key and the AWS secret keys have been successfully loaded from the configuration
- * and whether the S3 tests shall be performed.
- *
- * @return <code>true</code> if the tests shall be performed, <code>false</code> if the tests shall be skipped
- * because at least one AWS key is missing
- */
- private boolean testActivated() {
-
- final String accessKey = GlobalConfiguration.getString(S3FileSystem.S3_ACCESS_KEY_KEY, null);
- final String secretKey = GlobalConfiguration.getString(S3FileSystem.S3_SECRET_KEY_KEY, null);
-
- if (accessKey != null && secretKey != null) {
- return true;
- }
-
- return false;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/AbstractIDTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/AbstractIDTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/AbstractIDTest.java
deleted file mode 100644
index 2130f5b..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/AbstractIDTest.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-
-import org.junit.Test;
-
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.util.CommonTestUtils;
-
-/**
- * This class contains tests for the {@link AbstractID} class.
- *
- */
-public class AbstractIDTest {
-
- /**
- * Tests the setID method of an abstract ID.
- */
- @Test
- public void testSetID() {
-
- final ChannelID id1 = new ChannelID();
- final ChannelID id2 = new ChannelID();
- id1.setID(id2);
-
- assertEquals(id1.hashCode(), id2.hashCode());
- assertEquals(id1, id2);
- }
-
- /**
- * Tests the serialization/deserialization of an abstract ID.
- */
- @Test
- public void testSerialization() {
-
- final ChannelID origID = new ChannelID();
- try {
- final ChannelID copyID = (ChannelID) CommonTestUtils.createCopy(origID);
-
- assertEquals(origID.hashCode(), copyID.hashCode());
- assertEquals(origID, copyID);
-
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/DefaultChannelSelectorTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/DefaultChannelSelectorTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/DefaultChannelSelectorTest.java
deleted file mode 100644
index 387e49f..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/DefaultChannelSelectorTest.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/***********************************************************************************************************************
- * 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.io;
-
-import static org.junit.Assert.assertEquals;
-
-import org.junit.Test;
-
-import eu.stratosphere.core.io.StringRecord;
-
-/**
- * This class checks the functionality of the {@link DefaultChannelSelector} class.
- *
- */
-public class DefaultChannelSelectorTest {
-
- /**
- * This test checks the channel selection
- */
- @Test
- public void channelSelect() {
-
- final StringRecord dummyRecord = new StringRecord("abc");
- final DefaultChannelSelector<StringRecord> selector = new DefaultChannelSelector<StringRecord>();
- // Test with two channels
- final int numberOfOutputChannels = 2;
- int[] selectedChannels = selector.selectChannels(dummyRecord, numberOfOutputChannels);
- assertEquals(1, selectedChannels.length);
- assertEquals(1, selectedChannels[0]);
- selectedChannels = selector.selectChannels(dummyRecord, numberOfOutputChannels);
- assertEquals(1, selectedChannels.length);
- assertEquals(0, selectedChannels[0]);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/BufferTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/BufferTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/BufferTest.java
deleted file mode 100644
index 13cc30e..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/BufferTest.java
+++ /dev/null
@@ -1,192 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.nio.channels.FileChannel;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.types.IntegerRecord;
-
-/**
- * This class checks the functionality of the {@link SerializationBuffer} class and the {@link DefaultDeserializer}
- * class
- *
- */
-public class BufferTest
-{
- private File file = new File("./tmp");
-
- private FileInputStream fileinstream;
-
- private FileOutputStream filestream;
-
- private FileChannel writeable;
-
- private FileChannel readable;
-
- /**
- * Set up files and stream for testing
- *
- * @throws IOException
- */
- @Before
- public void before() throws IOException {
- file.createNewFile();
- filestream = new FileOutputStream(file);
- fileinstream = new FileInputStream(file);
- writeable = filestream.getChannel();
- readable = fileinstream.getChannel();
-
- }
-
- /**
- * clean up. Remove file close streams and channels
- *
- * @throws IOException
- */
- @After
- public void after() throws IOException {
- fileinstream.close();
- writeable.close();
- readable.close();
- filestream.close();
- file.delete();
- }
-
- /**
- * Tests serialization and deserialization of an {@link IntegerRecord}
- */
- @Test
- public void testIntSerialize()
- {
- final SerializationBuffer<IntegerRecord> intSerializationBuffer = new SerializationBuffer<IntegerRecord>();
- final int NUM = 0xab627ef;
-
- IntegerRecord intRecord = new IntegerRecord(NUM);
- // Serialize a record.
- try {
- intSerializationBuffer.serialize(intRecord);
- } catch (IOException e) {
- e.printStackTrace();
- fail();
- }
- // Last record is still in buffer, serializing another should throw IOException
- try {
- intSerializationBuffer.serialize(intRecord);
- fail();
- } catch (IOException e) {
- }
-
- // Read from buffer (written in file)
- try {
- intSerializationBuffer.read(writeable);
- } catch (IOException e) {
- e.printStackTrace();
- fail();
- }
- // Now a new Record can be serialized
- try {
- intSerializationBuffer.serialize(intRecord);
- } catch (IOException e) {
- e.printStackTrace();
- fail();
- }
-
- DefaultDeserializer<IntegerRecord> intDeserialitionBuffer = new DefaultDeserializer<IntegerRecord>(IntegerRecord.class, true);
- IntegerRecord record = new IntegerRecord();
- // Deserialize a Record
- try {
- record = intDeserialitionBuffer.readData(record, readable);
- } catch (IOException e) {
- e.printStackTrace();
- fail();
- }
- // Check it contains the right value
- assertEquals(NUM, record.getValue());
- // File empty, another read should throw IOException
- try {
- record = intDeserialitionBuffer.readData(record, readable);
- fail();
- } catch (IOException e) {
- }
-
- }
-
- /**
- * Tests serialization and deserialization of an {@link StringRecord}
- */
- @Test
- public void testStringSerialize()
- {
- final SerializationBuffer<StringRecord> stringSerializationBuffer = new SerializationBuffer<StringRecord>();
- final String str = "abc";
-
- StringRecord stringrecord = new StringRecord(str);
-
- // Serialize a record.
- try {
- stringSerializationBuffer.serialize(stringrecord);
- } catch (IOException e) {
- e.printStackTrace();
- fail();
- }
- // Read from buffer (write in file)
- try {
- stringSerializationBuffer.read(writeable);
- } catch (IOException e) {
- e.printStackTrace();
- fail();
- }
- // Serialize next Record.
- // Read from buffer (write in file)
- final String str2 = "abcdef";
- stringrecord = new StringRecord(str2);
- try {
- stringSerializationBuffer.serialize(stringrecord);
- stringSerializationBuffer.read(writeable);
- } catch (IOException e) {
- e.printStackTrace();
- fail();
- }
-
- final DefaultDeserializer<StringRecord> stringDeserialitionBuffer = new DefaultDeserializer<StringRecord>(StringRecord.class, true);
- StringRecord record = new StringRecord();
- // Deserialize and check record are correct
- try {
- record = stringDeserialitionBuffer.readData(record, readable);
- } catch (IOException e) {
- e.printStackTrace();
- fail();
- }
- assertEquals(str, record.toString());
- try {
- record = stringDeserialitionBuffer.readData(record, readable);
- } catch (IOException e) {
- e.printStackTrace();
- fail();
- }
- assertEquals(str2, record.toString());
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/MemoryBufferTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/MemoryBufferTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/MemoryBufferTest.java
deleted file mode 100644
index d8ffc27..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/MemoryBufferTest.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Queue;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import eu.stratosphere.core.memory.MemorySegment;
-import eu.stratosphere.nephele.util.BufferPoolConnector;
-
-
-public class MemoryBufferTest {
-
- private MemoryBufferPoolConnector bufferPoolConnector;
- private Queue<MemorySegment> bufferPool;
-
- private final static int INT_COUNT = 512;
- private final static int INT_SIZE = Integer.SIZE / Byte.SIZE;
-
- @Before
- public void setUp() throws Exception {
- bufferPool = new LinkedBlockingQueue<MemorySegment>();
- bufferPoolConnector = new BufferPoolConnector(bufferPool);
- }
-
- @After
- public void tearDown() throws Exception {
- }
-
- @Test
- public void readToSmallByteBuffer() throws IOException {
- MemoryBuffer buf = new MemoryBuffer(INT_COUNT*INT_SIZE, new MemorySegment(new byte[INT_COUNT*INT_SIZE]), bufferPoolConnector);
- fillBuffer(buf);
-
- ByteBuffer target = ByteBuffer.allocate(INT_SIZE);
- ByteBuffer largeTarget = ByteBuffer.allocate(INT_COUNT*INT_SIZE);
- int i = 0;
- while(buf.hasRemaining()) {
- buf.read(target);
- target.rewind();
- largeTarget.put(target);
- target.rewind();
- if( i++ >= INT_COUNT) {
- fail("There were too many elements in the buffer");
- }
- }
- assertEquals(-1, buf.read(target));
-
- target.rewind();
- validateByteBuffer(largeTarget);
- }
-
-
- /**
- * CopyToBuffer uses system.arraycopy()
- *
- * @throws IOException
- */
- @Test
- public void copyToBufferTest() throws IOException {
-
- MemoryBuffer buf = new MemoryBuffer(INT_COUNT*INT_SIZE, new MemorySegment(new byte[INT_COUNT*INT_SIZE]), bufferPoolConnector);
- fillBuffer(buf);
-
-
- // the target buffer is larger to check if the limit is set appropriately
- MemoryBuffer destination = new MemoryBuffer(INT_COUNT*INT_SIZE*2,
- new MemorySegment(new byte[INT_COUNT*INT_SIZE*2]),
- bufferPoolConnector);
- assertEquals(INT_COUNT*INT_SIZE*2, destination.limit());
- // copy buf contents to double sized MemBuffer
- buf.copyToBuffer(destination);
- assertEquals(INT_COUNT*INT_SIZE, destination.limit());
-
- // copy contents of destination to byteBuffer
- ByteBuffer test = ByteBuffer.allocate(INT_COUNT*INT_SIZE);
- int written = destination.read(test);
- assertEquals(INT_COUNT*INT_SIZE, written);
- // validate byteBuffer contents
- validateByteBuffer(test);
-
- destination.position(written);
- destination.limit(destination.getTotalSize());
- // allocate another byte buffer to write the rest of destination into a byteBuffer
- ByteBuffer testRemainder = ByteBuffer.allocate(INT_COUNT*INT_SIZE);
- written = destination.read(testRemainder);
- assertEquals(INT_COUNT*INT_SIZE, written);
- expectAllNullByteBuffer(testRemainder);
-
- buf.close(); // make eclipse happy
- }
-
- @Test
- public void testDuplicate() throws Exception {
- MemoryBuffer buf = new MemoryBuffer(INT_COUNT*INT_SIZE, new MemorySegment(new byte[INT_COUNT*INT_SIZE]), bufferPoolConnector);
- MemoryBuffer buf2 = buf.duplicate();
-
- buf2.close();
- buf.close();
- }
-
- private void fillBuffer(Buffer buf) throws IOException {
- ByteBuffer src = ByteBuffer.allocate(INT_SIZE);
- // write some data into buf:
- for(int i = 0; i < INT_COUNT; ++i) {
- src.putInt(0,i);
- src.rewind();
- buf.write(src);
- }
- buf.flip();
- }
-
-
- /**
- * Validates if the ByteBuffer contains the what fillMemoryBuffer has written!
- *
- * @param target
- */
- private void validateByteBuffer(ByteBuffer target) {
- ByteBuffer ref = ByteBuffer.allocate(INT_SIZE);
-
- for(int i = 0; i < INT_SIZE*INT_COUNT; ++i) {
- ref.putInt(0,i / INT_SIZE);
- assertEquals("Byte at position "+i+" is different", ref.get(i%INT_SIZE), target.get(i));
- }
- }
-
- private void expectAllNullByteBuffer(ByteBuffer target) {
- ByteBuffer ref = ByteBuffer.allocate(INT_SIZE);
- ref.putInt(0,0);
- for(int i = 0; i < INT_COUNT; ++i) {
- assertEquals("Byte at position "+i+" is different", ref.getInt(0), target.getInt(i));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/BooleanType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/BooleanType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/BooleanType.java
deleted file mode 100644
index 749a702..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/BooleanType.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class BooleanType implements SerializationTestType
-{
- private boolean value;
-
-
- public BooleanType()
- {
- this.value = false;
- }
-
- private BooleanType(boolean value)
- {
- this.value = value;
- }
-
-
- @Override
- public BooleanType getRandom(Random rnd)
- {
- return new BooleanType(rnd.nextBoolean());
- }
-
-
- @Override
- public void write(DataOutput out) throws IOException
- {
- out.writeBoolean(this.value);
- }
-
-
- @Override
- public void read(DataInput in) throws IOException
- {
- this.value = in.readBoolean();
- }
-
-
- @Override
- public int hashCode()
- {
- return this.value ? 1 : 0;
- }
-
-
- @Override
- public boolean equals(Object obj)
- {
- if (obj instanceof BooleanType) {
- BooleanType other = (BooleanType) obj;
- return this.value == other.value;
- } else {
- return false;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteArrayType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteArrayType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteArrayType.java
deleted file mode 100644
index 9c5bdca..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteArrayType.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Random;
-
-/**
- */
-public class ByteArrayType implements SerializationTestType
-{
- private static final int MAX_LEN = 512 * 15;
-
- private byte[] data;
-
-
- public ByteArrayType()
- {
- this.data = new byte[0];
- }
-
- private ByteArrayType(byte[] data)
- {
- this.data = data;
- }
-
-
- @Override
- public ByteArrayType getRandom(Random rnd)
- {
- final int len = rnd.nextInt(MAX_LEN) + 1;
- final byte[] data = new byte[len];
- rnd.nextBytes(data);
- return new ByteArrayType(data);
- }
-
-
-
-
- @Override
- public void write(DataOutput out) throws IOException
- {
- out.writeInt(this.data.length);
- out.write(this.data);
- }
-
-
- @Override
- public void read(DataInput in) throws IOException
- {
- final int len = in.readInt();
- this.data = new byte[len];
- in.readFully(this.data);
- }
-
-
- @Override
- public int hashCode()
- {
- return Arrays.hashCode(this.data);
- }
-
-
- @Override
- public boolean equals(Object obj)
- {
- if (obj instanceof ByteArrayType) {
- ByteArrayType other = (ByteArrayType) obj;
- return Arrays.equals(this.data, other.data);
- } else {
- return false;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteSubArrayType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteSubArrayType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteSubArrayType.java
deleted file mode 100644
index 719bd0d..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteSubArrayType.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Random;
-
-/**
- */
-public class ByteSubArrayType implements SerializationTestType
-{
- private static final int MAX_LEN = 512;
-
- private final byte[] data;
- private int len;
-
- public ByteSubArrayType()
- {
- this.data = new byte[MAX_LEN];
- this.len = 0;
- }
-
-
- @Override
- public ByteSubArrayType getRandom(Random rnd)
- {
- final int len = rnd.nextInt(MAX_LEN) + 1;
- final ByteSubArrayType t = new ByteSubArrayType();
- t.len = len;
-
- final byte[] data = t.data;
- for (int i = 0; i < len; i++) {
- data[i] = (byte) rnd.nextInt(256);
- }
-
- return t;
- }
-
-
- @Override
- public void write(DataOutput out) throws IOException
- {
- out.writeInt(this.len);
- out.write(this.data, 0, this.len);
- }
-
-
- @Override
- public void read(DataInput in) throws IOException
- {
- this.len = in.readInt();
- in.readFully(this.data, 0, this.len);
- }
-
-
- @Override
- public int hashCode()
- {
- final byte[] copy = new byte[this.len];
- System.arraycopy(this.data, 0, copy, 0, this.len);
- return Arrays.hashCode(copy);
- }
-
-
- @Override
- public boolean equals(Object obj)
- {
- if (obj instanceof ByteSubArrayType) {
- ByteSubArrayType other = (ByteSubArrayType) obj;
- if (this.len == other.len) {
- for (int i = 0; i < this.len; i++) {
- if (this.data[i] != other.data[i]) {
- return false;
- }
- }
- return true;
- } else {
- return false;
- }
- } else {
- return false;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteType.java
deleted file mode 100644
index 141f7b3..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ByteType.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class ByteType implements SerializationTestType
-{
- private byte value;
-
-
- public ByteType()
- {
- this.value = (byte) 0;
- }
-
- private ByteType(byte value)
- {
- this.value = value;
- }
-
-
- @Override
- public ByteType getRandom(Random rnd)
- {
- return new ByteType((byte) rnd.nextInt(256));
- }
-
-
- @Override
- public void write(DataOutput out) throws IOException
- {
- out.writeByte(this.value);
- }
-
-
- @Override
- public void read(DataInput in) throws IOException
- {
- this.value = in.readByte();
- }
-
-
- @Override
- public int hashCode()
- {
- return this.value;
- }
-
-
- @Override
- public boolean equals(Object obj)
- {
- if (obj instanceof ByteType) {
- ByteType other = (ByteType) obj;
- return this.value == other.value;
- } else {
- return false;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/CharType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/CharType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/CharType.java
deleted file mode 100644
index e51aeb8..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/CharType.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class CharType implements SerializationTestType
-{
- private char value;
-
-
- public CharType()
- {
- this.value = 0;
- }
-
- private CharType(char value)
- {
- this.value = value;
- }
-
-
- @Override
- public CharType getRandom(Random rnd)
- {
- return new CharType((char) rnd.nextInt(10000));
- }
-
-
- @Override
- public void write(DataOutput out) throws IOException
- {
- out.writeChar(this.value);
- }
-
-
- @Override
- public void read(DataInput in) throws IOException
- {
- this.value = in.readChar();
- }
-
-
- @Override
- public int hashCode()
- {
- return this.value;
- }
-
-
- @Override
- public boolean equals(Object obj)
- {
- if (obj instanceof CharType) {
- CharType other = (CharType) obj;
- return this.value == other.value;
- } else {
- return false;
- }
- }
-}
-
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DeSerializerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DeSerializerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DeSerializerTest.java
deleted file mode 100644
index 1c42277..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DeSerializerTest.java
+++ /dev/null
@@ -1,299 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.serialization;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.WritableByteChannel;
-import java.util.ArrayDeque;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-import java.util.Random;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import eu.stratosphere.nephele.io.channels.DefaultDeserializer;
-import eu.stratosphere.nephele.io.channels.SerializationBuffer;
-
-
-/**
- */
-public class DeSerializerTest
-{
- private static final SerializationTestType[] TYPE_FACTORIES = new SerializationTestType[] {
- new BooleanType(),
- new ByteArrayType(),
- new ByteSubArrayType(),
- new ByteType(),
- new CharType(),
- new DoubleType(),
- new FloatType(),
- new IntType(),
- new LongType(),
- new ShortType(),
- new UnsignedByteType(),
- new UnsignedShortType(),
- new UTFStringType()
- };
-
- private static final long SEED = 64871654635745873L;
-
- private Random rnd;
-
- // --------------------------------------------------------------------------------------------
-
- @Before
- public void setup()
- {
- this.rnd = new Random(SEED);
- }
-
- // --------------------------------------------------------------------------------------------
-
- @Test
- public void testSequenceOfIntegersWithAlignedBuffers()
- {
- try {
- final Random rnd = this.rnd;
- final int NUM_INTS = 1000000;
-
- final Iterator<SerializationTestType> intSource = new Iterator<SerializationTestType>()
- {
- private final Random random = rnd;
- private final int limit = NUM_INTS;
- private int pos = 0;
-
- @Override
- public boolean hasNext() {
- return this.pos < this.limit;
- }
- @Override
- public IntType next() {
- if (hasNext()) {
- this.pos++;
- return new IntType(this.random.nextInt());
- } else {
- throw new NoSuchElementException();
- }
- }
- @Override
- public void remove() {
- throw new UnsupportedOperationException();
- }
- };
-
- testSequenceOfTypes(intSource, 2048);
- }
- catch (Exception e) {
- e.printStackTrace();
- fail("Test encountered an unexpected exception.");
- }
- }
-
- @Test
- public void testSequenceOfIntegersWithUnalignedBuffers()
- {
- try {
- final Random rnd = this.rnd;
- final int NUM_INTS = 1000000;
-
- final Iterator<SerializationTestType> intSource = new Iterator<SerializationTestType>()
- {
- private final Random random = rnd;
- private final int limit = NUM_INTS;
- private int pos = 0;
-
- @Override
- public boolean hasNext() {
- return this.pos < this.limit;
- }
- @Override
- public IntType next() {
- if (hasNext()) {
- this.pos++;
- return new IntType(this.random.nextInt());
- } else {
- throw new NoSuchElementException();
- }
- }
- @Override
- public void remove() {
- throw new UnsupportedOperationException();
- }
- };
-
- testSequenceOfTypes(intSource, 2047);
- }
- catch (Exception e) {
- e.printStackTrace();
- fail("Test encountered an unexpected exception.");
- }
- }
-
- @Test
- public void testRandomTypes()
- {
- try {
- final Random rnd = this.rnd;
- final int NUM_TYPES = 1000000;
-
- final Iterator<SerializationTestType> randomSource = new Iterator<SerializationTestType>()
- {
- private final Random random = rnd;
- private final int limit = NUM_TYPES;
- private int pos = 0;
-
- @Override
- public boolean hasNext() {
- return this.pos < this.limit;
- }
- @Override
- public SerializationTestType next() {
- if (hasNext()) {
- this.pos++;
- return TYPE_FACTORIES[this.random.nextInt(TYPE_FACTORIES.length)].getRandom(this.random);
- } else {
- throw new NoSuchElementException();
- }
- }
- @Override
- public void remove() {
- throw new UnsupportedOperationException();
- }
- };
-
- // test with an odd buffer size to force many unaligned cases
- testSequenceOfTypes(randomSource, 512 * 7);
- }
- catch (Exception e) {
- e.printStackTrace();
- fail("Test encountered an unexpected exception.");
- }
- }
-
- private static final void testSequenceOfTypes(Iterator<SerializationTestType> sequence, int bufferSize) throws Exception
- {
- final ArrayDeque<SerializationTestType> elements = new ArrayDeque<SerializationTestType>(512);
-
- final PipeChannel channel = new PipeChannel(bufferSize);
- final SerializationBuffer<SerializationTestType> serBuffer = new SerializationBuffer<SerializationTestType>();
- final DefaultDeserializer<SerializationTestType> deserBuffer = new DefaultDeserializer<SerializationTestType>(null);
-
- while (sequence.hasNext()) {
- final SerializationTestType type = sequence.next();
-
- // serialize the record
- serBuffer.serialize(type);
- elements.addLast(type);
-
- // write the serialized record
- while (true) {
- serBuffer.read(channel);
- if (serBuffer.dataLeftFromPreviousSerialization()) {
- // current buffer is full, we need to start de-serializing to make space
- channel.flip();
-
- while (!elements.isEmpty()) {
- final SerializationTestType reference = elements.pollFirst();
- final SerializationTestType result = deserBuffer.readData(reference.getClass().newInstance(), channel);
- if (result == null) {
- // not yet complete, we need to break
- elements.addFirst(reference);
- break;
- } else {
- // validate that we deserialized correctly
- assertEquals("The deserialized element is not equal to the serialized element.", reference, result);
- }
- }
-
- channel.clear();
- } else {
- break;
- }
- }
- }
-
- // check the remaining records in the buffers...
- channel.flip();
- while (!elements.isEmpty()) {
- final SerializationTestType reference = elements.pollFirst();
- final SerializationTestType result = deserBuffer.readData(reference.getClass().newInstance(), channel);
-
- assertNotNull(result);
- assertEquals("The deserialized element is not equal to the serialized element.", reference, result);
- }
- }
-
- // ============================================================================================
-
- private static final class PipeChannel implements WritableByteChannel, ReadableByteChannel
- {
- private final byte[] buffer;
-
- private int position;
- private int limit;
-
-
- PipeChannel(int capacity) {
- this.buffer = new byte[capacity];
- this.limit = capacity;
- }
-
- public void flip() {
- this.limit = this.position;
- this.position = 0;
- }
-
- public void clear() {
- this.position = 0;
- this.limit = this.buffer.length;
- }
-
- @Override
- public boolean isOpen() {
- return true;
- }
-
- @Override
- public void close()
- {}
-
-
- @Override
- public int write(ByteBuffer src)
- {
- final int toGet = Math.min(this.limit - this.position, src.remaining());
- src.get(this.buffer, this.position, toGet);
- this.position += toGet;
- return toGet;
- }
-
-
- @Override
- public int read(ByteBuffer dst) throws IOException
- {
- final int toPut = Math.min(this.limit - this.position, dst.remaining());
- dst.put(this.buffer, this.position, toPut);
- this.position += toPut;
- return toPut;
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DoubleType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DoubleType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DoubleType.java
deleted file mode 100644
index 3f3e958..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/DoubleType.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class DoubleType implements SerializationTestType
-{
- private double value;
-
-
- public DoubleType()
- {
- this.value = 0;
- }
-
- private DoubleType(double value)
- {
- this.value = value;
- }
-
-
- @Override
- public DoubleType getRandom(Random rnd)
- {
- return new DoubleType(rnd.nextDouble());
- }
-
-
- @Override
- public void write(DataOutput out) throws IOException
- {
- out.writeDouble(this.value);
- }
-
-
- @Override
- public void read(DataInput in) throws IOException
- {
- this.value = in.readDouble();
- }
-
-
- @Override
- public int hashCode()
- {
- final long l = Double.doubleToLongBits(this.value);
- return (int) (l ^ l >>> 32);
- }
-
-
- @Override
- public boolean equals(Object obj)
- {
- if (obj instanceof DoubleType) {
- DoubleType other = (DoubleType) obj;
- return Double.doubleToLongBits(this.value) == Double.doubleToLongBits(other.value);
- } else {
- return false;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/FloatType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/FloatType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/FloatType.java
deleted file mode 100644
index 7bc37a0..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/FloatType.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class FloatType implements SerializationTestType
-{
- private float value;
-
-
- public FloatType()
- {
- this.value = 0;
- }
-
- private FloatType(float value)
- {
- this.value = value;
- }
-
-
- @Override
- public FloatType getRandom(Random rnd)
- {
- return new FloatType(rnd.nextFloat());
- }
-
-
- @Override
- public void write(DataOutput out) throws IOException
- {
- out.writeFloat(this.value);
- }
-
-
- @Override
- public void read(DataInput in) throws IOException
- {
- this.value = in.readFloat();
- }
-
-
- @Override
- public int hashCode()
- {
- return Float.floatToIntBits(this.value);
- }
-
-
- @Override
- public boolean equals(Object obj)
- {
- if (obj instanceof FloatType) {
- FloatType other = (FloatType) obj;
- return Float.floatToIntBits(this.value) == Float.floatToIntBits(other.value);
- } else {
- return false;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/IntType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/IntType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/IntType.java
deleted file mode 100644
index 1180884..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/IntType.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class IntType implements SerializationTestType
-{
- private int value;
-
-
- public IntType()
- {
- this.value = 0;
- }
-
- public IntType(int value)
- {
- this.value = value;
- }
-
-
- @Override
- public IntType getRandom(Random rnd)
- {
- return new IntType(rnd.nextInt());
- }
-
-
- @Override
- public void write(DataOutput out) throws IOException
- {
- out.writeInt(this.value);
- }
-
-
- @Override
- public void read(DataInput in) throws IOException
- {
- this.value = in.readInt();
- }
-
-
- @Override
- public int hashCode()
- {
- return this.value;
- }
-
-
- @Override
- public boolean equals(Object obj)
- {
- if (obj instanceof IntType) {
- IntType other = (IntType) obj;
- return this.value == other.value;
- } else {
- return false;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/LongType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/LongType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/LongType.java
deleted file mode 100644
index 8c752f6..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/LongType.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class LongType implements SerializationTestType
-{
- private long value;
-
-
- public LongType()
- {
- this.value = 0;
- }
-
- private LongType(long value)
- {
- this.value = value;
- }
-
-
- @Override
- public LongType getRandom(Random rnd)
- {
- return new LongType(rnd.nextLong());
- }
-
-
- @Override
- public void write(DataOutput out) throws IOException
- {
- out.writeLong(this.value);
- }
-
-
- @Override
- public void read(DataInput in) throws IOException
- {
- this.value = in.readLong();
- }
-
-
- @Override
- public int hashCode()
- {
- return (int) (this.value ^ this.value >>> 32);
- }
-
-
- @Override
- public boolean equals(Object obj)
- {
- if (obj instanceof LongType) {
- LongType other = (LongType) obj;
- return this.value == other.value;
- } else {
- return false;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/SerializationTestType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/SerializationTestType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/SerializationTestType.java
deleted file mode 100644
index 77630b0..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/SerializationTestType.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.serialization;
-
-import java.util.Random;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-
-/**
- */
-public interface SerializationTestType extends IOReadableWritable
-{
- public SerializationTestType getRandom(Random rnd);
-
- // public static final String REST1_PATH = "src/test/resources/clustering/rest1.json";
- // public static final String SAMPLE1_PATH = "src/test/resources/clustering/sample1.json";
- //
- // public static IJsonNode asJson(final String... values) {
- // return new Point(String.valueOf(pointCount++), values).write(null);
- // }
- //
- // public static List<IJsonNode> loadPoints(final String filePath)
- // throws IOException {
- // BufferedReader reader = null;
- // try {
- // final File pointFile = new File(filePath);
- // reader = new BufferedReader(new FileReader(pointFile));
- // final JsonParser parser = new JsonParser(reader);
- // final List<IJsonNode> pointNodes = new LinkedList<IJsonNode>();
- // while (!parser.checkEnd())
- // pointNodes.add(parser.readValueAsTree());
- // return pointNodes;
- // } finally {
- // try {
- // reader.close();
- // } catch (final Exception e) {
- // e.printStackTrace();
- // }
- // }
- // }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ShortType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ShortType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ShortType.java
deleted file mode 100644
index 09d0009..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/ShortType.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class ShortType implements SerializationTestType
-{
- private short value;
-
-
- public ShortType()
- {
- this.value = (short) 0;
- }
-
- private ShortType(short value)
- {
- this.value = value;
- }
-
-
- @Override
- public ShortType getRandom(Random rnd)
- {
- return new ShortType((short) rnd.nextInt(65536));
- }
-
-
- @Override
- public void write(DataOutput out) throws IOException
- {
- out.writeShort(this.value);
- }
-
-
- @Override
- public void read(DataInput in) throws IOException
- {
- this.value = in.readShort();
- }
-
-
- @Override
- public int hashCode()
- {
- return this.value;
- }
-
-
- @Override
- public boolean equals(Object obj)
- {
- if (obj instanceof ShortType) {
- ShortType other = (ShortType) obj;
- return this.value == other.value;
- } else {
- return false;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UTFStringType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UTFStringType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UTFStringType.java
deleted file mode 100644
index c4481fa..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UTFStringType.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class UTFStringType implements SerializationTestType
-{
- private static final int MAX_LEN = 1500;
-
- private String value;
-
-
- public UTFStringType()
- {
- this.value = "";
- }
-
- private UTFStringType(String value)
- {
- this.value = value;
- }
-
-
- @Override
- public UTFStringType getRandom(Random rnd)
- {
- final StringBuilder bld = new StringBuilder();
- final int len = rnd.nextInt(MAX_LEN + 1);
-
- for (int i = 0; i < len; i++) {
- bld.append((char) rnd.nextInt(Character.MAX_VALUE));
- }
-
- return new UTFStringType(bld.toString());
- }
-
-
- @Override
- public void write(DataOutput out) throws IOException
- {
- out.writeUTF(this.value);
- }
-
-
- @Override
- public void read(DataInput in) throws IOException
- {
- this.value = in.readUTF();
- }
-
-
- @Override
- public int hashCode()
- {
- return this.value.hashCode();
- }
-
-
- @Override
- public boolean equals(Object obj)
- {
- if (obj instanceof UTFStringType) {
- UTFStringType other = (UTFStringType) obj;
- return this.value.equals(other.value);
- } else {
- return false;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedByteType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedByteType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedByteType.java
deleted file mode 100644
index 4edcd05..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedByteType.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class UnsignedByteType implements SerializationTestType
-{
- private int value;
-
-
- public UnsignedByteType()
- {
- this.value = 0;
- }
-
- private UnsignedByteType(int value)
- {
- this.value = value;
- }
-
-
- @Override
- public UnsignedByteType getRandom(Random rnd)
- {
- return new UnsignedByteType(rnd.nextInt(128) + 128);
- }
-
-
- @Override
- public void write(DataOutput out) throws IOException
- {
- out.writeByte(this.value);
- }
-
-
- @Override
- public void read(DataInput in) throws IOException
- {
- this.value = in.readUnsignedByte();
- }
-
-
- @Override
- public int hashCode()
- {
- return this.value;
- }
-
-
- @Override
- public boolean equals(Object obj)
- {
- if (obj instanceof UnsignedByteType) {
- UnsignedByteType other = (UnsignedByteType) obj;
- return this.value == other.value;
- } else {
- return false;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedShortType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedShortType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedShortType.java
deleted file mode 100644
index 898b343..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/channels/serialization/UnsignedShortType.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.channels.serialization;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Random;
-
-/**
- */
-public class UnsignedShortType implements SerializationTestType
-{
- private int value;
-
-
- public UnsignedShortType()
- {
- this.value = 0;
- }
-
- private UnsignedShortType(int value)
- {
- this.value = value;
- }
-
-
- @Override
- public UnsignedShortType getRandom(Random rnd)
- {
- return new UnsignedShortType(rnd.nextInt(32768) + 32768);
- }
-
-
- @Override
- public void write(DataOutput out) throws IOException
- {
- out.writeShort(this.value);
- }
-
-
- @Override
- public void read(DataInput in) throws IOException
- {
- this.value = in.readUnsignedShort();
- }
-
-
- @Override
- public int hashCode()
- {
- return this.value;
- }
-
-
- @Override
- public boolean equals(Object obj)
- {
- if (obj instanceof UnsignedShortType) {
- UnsignedShortType other = (UnsignedShortType) obj;
- return this.value == other.value;
- } else {
- return false;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/library/FileLineReadWriteTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/library/FileLineReadWriteTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/library/FileLineReadWriteTest.java
deleted file mode 100644
index f88d7bf..0000000
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/io/library/FileLineReadWriteTest.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.library;
-
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-import static org.powermock.api.mockito.PowerMockito.whenNew;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.Mock;
-import org.mockito.MockitoAnnotations;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-import org.powermock.reflect.Whitebox;
-
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.core.fs.FileInputSplit;
-import eu.stratosphere.core.fs.Path;
-import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.execution.Environment;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.InputSplitProvider;
-
-/**
- * This class checks the functionality of the {@link FileLineReader} and the {@link FileLineWriter} class.
- *
- */
-@RunWith(PowerMockRunner.class)
-@PrepareForTest(FileLineReader.class)
-public class FileLineReadWriteTest {
-
- @Mock
- private Environment environment;
-
- @Mock
- private Configuration conf;
-
- @Mock
- private RecordReader<StringRecord> recordReader;
-
- @Mock
- private RecordWriter<StringRecord> recordWriter;
-
- @Mock
- private InputSplitProvider inputSplitProvider;
-
- private File file = new File("./tmp");
-
- /**
- * Set up mocks
- *
- * @throws IOException
- */
- @Before
- public void before() throws Exception {
-
- MockitoAnnotations.initMocks(this);
- }
-
- /**
- * remove the temporary file
- */
- @After
- public void after() {
- this.file.delete();
- }
-
- /**
- * Tests the read and write methods
- *
- * @throws Exception
- */
- @Test
- public void testReadWrite() throws Exception {
-
- this.file.createNewFile();
- FileLineWriter writer = new FileLineWriter();
- Whitebox.setInternalState(writer, "environment", this.environment);
- Whitebox.setInternalState(writer, "input", this.recordReader);
- when(this.environment.getTaskConfiguration()).thenReturn(this.conf);
-
- when(this.conf.getString("outputPath", null)).thenReturn(this.file.toURI().toString());
- when(this.recordReader.hasNext()).thenReturn(true, true, true, false);
- StringRecord in = new StringRecord("abc");
- try {
- when(this.recordReader.next()).thenReturn(in);
- } catch (IOException e) {
- fail();
- e.printStackTrace();
- } catch (InterruptedException e) {
- fail();
- e.printStackTrace();
- }
- writer.invoke();
-
- final FileInputSplit split = new FileInputSplit(0, new Path(this.file.toURI().toString()), 0,
- this.file.length(), null);
- when(this.environment.getInputSplitProvider()).thenReturn(this.inputSplitProvider);
- when(this.inputSplitProvider.getNextInputSplit()).thenReturn(split, (FileInputSplit) null);
-
- FileLineReader reader = new FileLineReader();
- Whitebox.setInternalState(reader, "environment", this.environment);
- Whitebox.setInternalState(reader, "output", this.recordWriter);
- StringRecord record = mock(StringRecord.class);
-
- whenNew(StringRecord.class).withNoArguments().thenReturn(record);
-
- reader.invoke();
-
- // verify the correct bytes have been written and read
- verify(record, times(3)).set(in.getBytes());
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleSourceTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleSourceTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleSourceTask.java
index 6d29521..1e2be47 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleSourceTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleSourceTask.java
@@ -19,7 +19,7 @@ import eu.stratosphere.core.fs.FSDataInputStream;
import eu.stratosphere.core.fs.FileInputSplit;
import eu.stratosphere.core.fs.FileSystem;
import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.nephele.template.AbstractFileInputTask;
import eu.stratosphere.runtime.fs.LineReader;
@@ -31,6 +31,8 @@ public class DoubleSourceTask extends AbstractFileInputTask {
@Override
public void invoke() throws Exception {
+ this.output1.initializeSerializers();
+ this.output2.initializeSerializers();
final Iterator<FileInputSplit> splitIterator = getFileInputSplits();
@@ -65,12 +67,15 @@ public class DoubleSourceTask extends AbstractFileInputTask {
// Close the stream;
lineReader.close();
}
+
+ this.output1.flush();
+ this.output2.flush();
}
@Override
public void registerInputOutput() {
- this.output1 = new RecordWriter<StringRecord>(this, StringRecord.class);
- this.output2 = new RecordWriter<StringRecord>(this, StringRecord.class);
+ this.output1 = new RecordWriter<StringRecord>(this);
+ this.output2 = new RecordWriter<StringRecord>(this);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleTargetTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleTargetTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleTargetTask.java
index cea9dd2..f0ca435 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleTargetTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/DoubleTargetTask.java
@@ -14,8 +14,8 @@
package eu.stratosphere.nephele.jobmanager;
import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.nephele.template.AbstractTask;
public class DoubleTargetTask extends AbstractTask {
@@ -29,6 +29,8 @@ public class DoubleTargetTask extends AbstractTask {
@Override
public void invoke() throws Exception {
+ this.output.initializeSerializers();
+
while (this.input1.hasNext()) {
StringRecord s = input1.next();
@@ -41,13 +43,15 @@ public class DoubleTargetTask extends AbstractTask {
this.output.emit(s);
}
+ this.output.flush();
+
}
@Override
public void registerInputOutput() {
this.input1 = new RecordReader<StringRecord>(this, StringRecord.class);
this.input2 = new RecordReader<StringRecord>(this, StringRecord.class);
- this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
+ this.output = new RecordWriter<StringRecord>(this);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionTask.java
index baae3b9..77b4f96 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ExceptionTask.java
@@ -14,8 +14,8 @@
package eu.stratosphere.nephele.jobmanager;
import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.nephele.template.AbstractTask;
/**
@@ -57,7 +57,7 @@ public class ExceptionTask extends AbstractTask {
public void registerInputOutput() {
new RecordReader<StringRecord>(this, StringRecord.class);
- new RecordWriter<StringRecord>(this, StringRecord.class);
+ new RecordWriter<StringRecord>(this);
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ForwardTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ForwardTask.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ForwardTask.java
index be7be66..96be668 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ForwardTask.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/jobmanager/ForwardTask.java
@@ -14,8 +14,8 @@
package eu.stratosphere.nephele.jobmanager;
import eu.stratosphere.core.io.StringRecord;
-import eu.stratosphere.nephele.io.RecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.nephele.template.AbstractTask;
public class ForwardTask extends AbstractTask {
@@ -26,16 +26,20 @@ public class ForwardTask extends AbstractTask {
@Override
public void invoke() throws Exception {
+ this.output.initializeSerializers();
+
while (this.input.hasNext()) {
StringRecord s = input.next();
this.output.emit(s);
}
+
+ this.output.flush();
}
@Override
public void registerInputOutput() {
this.input = new RecordReader<StringRecord>(this, StringRecord.class);
- this.output = new RecordWriter<StringRecord>(this, StringRecord.class);
+ this.output = new RecordWriter<StringRecord>(this);
}
}
[17/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java
deleted file mode 100644
index c1553d8..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputChannelContext.java
+++ /dev/null
@@ -1,303 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.IOException;
-import java.util.ArrayDeque;
-import java.util.Iterator;
-import java.util.Queue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel;
-import eu.stratosphere.nephele.io.channels.bytebuffered.BufferOrEvent;
-import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedInputChannelBroker;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferAvailabilityListener;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.InputChannelContext;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ReceiverNotFoundEvent;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher;
-
-
-final class RuntimeInputChannelContext implements InputChannelContext, ByteBufferedInputChannelBroker {
-
- private static final Log LOG = LogFactory.getLog(RuntimeInputChannelContext.class);
-
- private final RuntimeInputGateContext inputGateContext;
-
- private final AbstractByteBufferedInputChannel<?> byteBufferedInputChannel;
-
- private final TransferEnvelopeDispatcher transferEnvelopeDispatcher;
-
- private final Queue<TransferEnvelope> queuedEnvelopes = new ArrayDeque<TransferEnvelope>();
-
- private Iterator<AbstractEvent> pendingEvents;
-
- private int lastReceivedEnvelope = -1;
-
- private boolean destroyCalled = false;
-
- RuntimeInputChannelContext(final RuntimeInputGateContext inputGateContext,
- final TransferEnvelopeDispatcher transferEnvelopeDispatcher,
- final AbstractByteBufferedInputChannel<?> byteBufferedInputChannel) {
-
- this.inputGateContext = inputGateContext;
- this.transferEnvelopeDispatcher = transferEnvelopeDispatcher;
- this.byteBufferedInputChannel = byteBufferedInputChannel;
- this.byteBufferedInputChannel.setInputChannelBroker(this);
- }
-
-
- @Override
- public BufferOrEvent getNextBufferOrEvent() throws IOException {
- // return pending events first
- if (this.pendingEvents != null) {
- // if the field is not null, it must always have a next value!
- BufferOrEvent next = new BufferOrEvent(this.pendingEvents.next());
- if (!this.pendingEvents.hasNext()) {
- this.pendingEvents = null;
- }
- return next;
- }
-
- // if no events are pending, get the next buffer
- TransferEnvelope nextEnvelope;
- synchronized (this.queuedEnvelopes) {
- if (this.queuedEnvelopes.isEmpty()) {
- return null;
- }
- nextEnvelope = this.queuedEnvelopes.poll();
- }
-
- // schedule events as pending, because events come always after the buffer!
- if (nextEnvelope.getEventList() != null) {
- Iterator<AbstractEvent> events = nextEnvelope.getEventList().iterator();
- if (events.hasNext()) {
- this.pendingEvents = events;
- }
- }
-
- // get the buffer, if there is one
- if (nextEnvelope.getBuffer() != null) {
- return new BufferOrEvent(nextEnvelope.getBuffer());
- }
- else if (this.pendingEvents != null) {
- // if the field is not null, it must always have a next value!
- BufferOrEvent next = new BufferOrEvent(this.pendingEvents.next());
- if (!this.pendingEvents.hasNext()) {
- this.pendingEvents = null;
- }
-
- return next;
- }
- else {
- // no buffer and no events, this should be an error
- throw new IOException("Received an envelope with neither data nor events.");
- }
- }
-
- @Override
- public void transferEventToOutputChannel(AbstractEvent event) throws IOException, InterruptedException {
- TransferEnvelope ephemeralTransferEnvelope = new TransferEnvelope(0, getJobID(), getChannelID());
- ephemeralTransferEnvelope.addEvent(event);
-
- this.transferEnvelopeDispatcher.processEnvelopeFromInputChannel(ephemeralTransferEnvelope);
- }
-
- @Override
- public void queueTransferEnvelope(TransferEnvelope transferEnvelope) {
-
- if (ReceiverNotFoundEvent.isReceiverNotFoundEvent(transferEnvelope)) {
- return;
- }
-
- // The sequence number of the envelope to be queued
- final int sequenceNumber = transferEnvelope.getSequenceNumber();
-
- synchronized (this.queuedEnvelopes) {
-
- if (this.destroyCalled) {
- final Buffer buffer = transferEnvelope.getBuffer();
- if (buffer != null) {
- buffer.recycleBuffer();
- }
- return;
- }
-
- final int expectedSequenceNumber = this.lastReceivedEnvelope + 1;
- if (sequenceNumber != expectedSequenceNumber) {
- // This is a problem, now we are actually missing some data
- this.byteBufferedInputChannel.reportIOException(new IOException("Expected data packet "
- + expectedSequenceNumber + " but received " + sequenceNumber));
-
- // notify that something (an exception) is available
- this.byteBufferedInputChannel.notifyGateThatInputIsAvailable();
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Input channel " + getChannelName() + " expected envelope " + expectedSequenceNumber
- + " but received " + sequenceNumber);
- }
-
- // rescue the buffer
- final Buffer buffer = transferEnvelope.getBuffer();
- if (buffer != null) {
- buffer.recycleBuffer();
- }
- } else {
-
- this.queuedEnvelopes.add(transferEnvelope);
- this.lastReceivedEnvelope = sequenceNumber;
-
- // Notify the channel about the new data. notify as much as there is (buffer plus once per event)
- if (transferEnvelope.getBuffer() != null) {
- this.byteBufferedInputChannel.notifyGateThatInputIsAvailable();
- }
- if (transferEnvelope.getEventList() != null) {
- for (int i = 0; i < transferEnvelope.getEventList().size(); i++) {
- this.byteBufferedInputChannel.notifyGateThatInputIsAvailable();
- }
- }
- }
- }
- }
-
- @Override
- public ChannelID getChannelID() {
- return this.byteBufferedInputChannel.getID();
- }
-
- @Override
- public ChannelID getConnectedChannelID() {
- return this.byteBufferedInputChannel.getConnectedChannelID();
- }
-
- @Override
- public JobID getJobID() {
- return this.byteBufferedInputChannel.getJobID();
- }
-
- @Override
- public boolean isInputChannel() {
- return this.byteBufferedInputChannel.isInputChannel();
- }
-
- @Override
- public void destroy() {
- final Queue<Buffer> buffersToRecycle = new ArrayDeque<Buffer>();
-
- synchronized (this.queuedEnvelopes) {
- this.destroyCalled = true;
-
- while (!this.queuedEnvelopes.isEmpty()) {
- final TransferEnvelope envelope = this.queuedEnvelopes.poll();
- if (envelope.getBuffer() != null) {
- buffersToRecycle.add(envelope.getBuffer());
- }
- }
- }
-
- while (!buffersToRecycle.isEmpty()) {
- buffersToRecycle.poll().recycleBuffer();
- }
- }
-
- @Override
- public void logQueuedEnvelopes() {
- int numberOfQueuedEnvelopes = 0;
- int numberOfQueuedEnvelopesWithMemoryBuffers = 0;
- int numberOfQueuedEnvelopesWithFileBuffers = 0;
-
- synchronized (this.queuedEnvelopes) {
-
- final Iterator<TransferEnvelope> it = this.queuedEnvelopes.iterator();
- while (it.hasNext()) {
-
- final TransferEnvelope envelope = it.next();
- ++numberOfQueuedEnvelopes;
- final Buffer buffer = envelope.getBuffer();
- if (buffer == null) {
- continue;
- }
-
- if (buffer.isBackedByMemory()) {
- ++numberOfQueuedEnvelopesWithMemoryBuffers;
- } else {
- ++numberOfQueuedEnvelopesWithFileBuffers;
- }
- }
- }
-
- System.out.println("\t\t" + getChannelName() + ": " + numberOfQueuedEnvelopes + " ("
- + numberOfQueuedEnvelopesWithMemoryBuffers + ", " + numberOfQueuedEnvelopesWithFileBuffers + ")");
-
- }
-
- @Override
- public Buffer requestEmptyBuffer(int minimumSizeOfBuffer) throws IOException {
- return this.inputGateContext.requestEmptyBuffer(minimumSizeOfBuffer);
- }
-
- @Override
- public Buffer requestEmptyBufferBlocking(int minimumSizeOfBuffer) throws IOException, InterruptedException {
- return this.inputGateContext.requestEmptyBufferBlocking(minimumSizeOfBuffer);
- }
-
- @Override
- public int getMaximumBufferSize() {
- return this.inputGateContext.getMaximumBufferSize();
- }
-
- @Override
- public boolean isShared() {
- return this.inputGateContext.isShared();
- }
-
- @Override
- public void reportAsynchronousEvent() {
- this.inputGateContext.reportAsynchronousEvent();
- }
-
- @Override
- public ChannelType getType() {
- return this.byteBufferedInputChannel.getType();
- }
-
- /**
- * Constructs and returns a human-readable name of this channel used for debugging.
- *
- * @return a human-readable name of this channel used for debugging
- */
- private String getChannelName() {
- StringBuilder sb = new StringBuilder(this.inputGateContext.getTaskName());
- sb.append(' ');
- sb.append('(');
- sb.append(this.byteBufferedInputChannel.getChannelIndex());
- sb.append(',');
- sb.append(' ');
- sb.append(this.byteBufferedInputChannel.getID());
- sb.append(')');
- return sb.toString();
- }
-
- @Override
- public boolean registerBufferAvailabilityListener(final BufferAvailabilityListener bufferAvailabilityListener) {
- return this.inputGateContext.registerBufferAvailabilityListener(bufferAvailabilityListener);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java
deleted file mode 100644
index 7e2d492..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeInputGateContext.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.channels.AbstractInputChannel;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedInputChannel;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferAvailabilityListener;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.InputChannelContext;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.InputGateContext;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher;
-
-final class RuntimeInputGateContext implements BufferProvider, InputGateContext, LocalBufferPoolOwner {
-
- private final String taskName;
-
- private final LocalBufferPool localBufferPool;
-
- private final TransferEnvelopeDispatcher transferEnvelopeDispatcher;
-
- private final InputGate<? extends IOReadableWritable> inputGate;
-
- RuntimeInputGateContext(final String taskName, final TransferEnvelopeDispatcher transferEnvelopeDispatcher,
- final InputGate<? extends IOReadableWritable> inputGate) {
-
- this.taskName = taskName;
- this.localBufferPool = new LocalBufferPool(1, false);
-
- this.transferEnvelopeDispatcher = transferEnvelopeDispatcher;
- this.inputGate = inputGate;
- }
-
- @Override
- public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException {
-
- return this.localBufferPool.requestEmptyBuffer(minimumSizeOfBuffer);
- }
-
-
- @Override
- public Buffer requestEmptyBufferBlocking(final int minimumSizeOfBuffer) throws IOException, InterruptedException {
-
- final Buffer buffer = this.localBufferPool.requestEmptyBuffer(minimumSizeOfBuffer);
- if (buffer != null) {
- return buffer;
- }
-
- return this.localBufferPool.requestEmptyBufferBlocking(minimumSizeOfBuffer);
- }
-
-
- @Override
- public int getMaximumBufferSize() {
-
- return this.localBufferPool.getMaximumBufferSize();
- }
-
-
- @Override
- public boolean isShared() {
-
- return this.localBufferPool.isShared();
- }
-
-
- @Override
- public void reportAsynchronousEvent() {
-
- this.localBufferPool.reportAsynchronousEvent();
- }
-
- @Override
- public int getNumberOfChannels() {
-
- return this.inputGate.getNumberOfInputChannels();
- }
-
-
- @Override
- public void setDesignatedNumberOfBuffers(int numberOfBuffers) {
-
- this.localBufferPool.setDesignatedNumberOfBuffers(numberOfBuffers);
- }
-
-
- @Override
- public void clearLocalBufferPool() {
-
- this.localBufferPool.destroy();
- }
-
-
- @Override
- public void logBufferUtilization() {
-
- final int ava = this.localBufferPool.getNumberOfAvailableBuffers();
- final int req = this.localBufferPool.getRequestedNumberOfBuffers();
- final int des = this.localBufferPool.getDesignatedNumberOfBuffers();
-
- System.out
- .println("\t\tInput gate " + this.inputGate.getIndex() + " of " + this.taskName + ": " + ava
- + " available, " + req + " requested, " + des + " designated");
- }
-
-
- @Override
- public GateID getGateID() {
-
- return this.inputGate.getGateID();
- }
-
-
- @Override
- public InputChannelContext createInputChannelContext(final ChannelID channelID,
- final InputChannelContext previousContext) {
-
- AbstractInputChannel<? extends IOReadableWritable> channel = null;
- for (int i = 0; i < this.inputGate.getNumberOfInputChannels(); ++i) {
- AbstractInputChannel<? extends IOReadableWritable> candidateChannel = this.inputGate.getInputChannel(i);
- if (candidateChannel.getID().equals(channelID)) {
- channel = candidateChannel;
- break;
- }
- }
-
- if (channel == null) {
- throw new IllegalArgumentException("Cannot find input channel with ID " + channelID);
- }
-
- if (!(channel instanceof AbstractByteBufferedInputChannel)) {
- throw new IllegalStateException("Channel with ID" + channelID
- + " is not of type AbstractByteBufferedInputChannel");
- }
-
- return new RuntimeInputChannelContext(this, this.transferEnvelopeDispatcher,
- (AbstractByteBufferedInputChannel<? extends IOReadableWritable>) channel);
- }
-
-
- @Override
- public LocalBufferPoolOwner getLocalBufferPoolOwner() {
-
- return this;
- }
-
- /**
- * Returns the name of the task this gate belongs to.
- *
- * @return the name of the task this gate belongs to
- */
- String getTaskName() {
-
- return this.taskName;
- }
-
-
- @Override
- public boolean registerBufferAvailabilityListener(final BufferAvailabilityListener bufferAvailabilityListener) {
-
- return this.localBufferPool.registerBufferAvailabilityListener(bufferAvailabilityListener);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java
deleted file mode 100644
index 41fc7ff..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelBroker.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.IOException;
-
-import eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel;
-import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedChannelCloseEvent;
-import eu.stratosphere.nephele.io.channels.bytebuffered.ByteBufferedOutputChannelBroker;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.ReceiverNotFoundEvent;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelope;
-
-final class RuntimeOutputChannelBroker extends AbstractOutputChannelForwarder implements
- ByteBufferedOutputChannelBroker {
-
- /**
- * The byte buffered output channel this context belongs to.
- */
- private final AbstractByteBufferedOutputChannel<?> byteBufferedOutputChannel;
-
- /**
- * The buffer provider this channel broker to obtain buffers from.
- */
- private final RuntimeOutputGateContext outputGateContext;
-
- /**
- * The forwarding chain along which the created transfer envelopes will be pushed.
- */
- private OutputChannelForwardingChain forwardingChain;
-
- /**
- * Points to the {@link TransferEnvelope} object that will be passed to the framework upon
- * the next <code>releaseWriteBuffers</code> call.
- */
- private TransferEnvelope outgoingTransferEnvelope = null;
-
- /**
- * Stores whether the receiver has acknowledged the close request from this channel.
- */
- private boolean closeAcknowledgmentReceived = false;
-
- /**
- * Stores the last sequence number of the transfer envelope for which the receiver could not be found.
- */
- private int lastSequenceNumberWithReceiverNotFound = -1;
-
- /**
- * The sequence number for the next {@link TransferEnvelope} to be created.
- */
- private int sequenceNumber = 0;
-
- RuntimeOutputChannelBroker(final RuntimeOutputGateContext outputGateContext,
- final AbstractByteBufferedOutputChannel<?> byteBufferedOutputChannel,
- final AbstractOutputChannelForwarder next) {
-
- super(next);
-
- if (next == null) {
- throw new IllegalArgumentException("Argument next must not be null");
- }
-
- this.outputGateContext = outputGateContext;
- this.byteBufferedOutputChannel = byteBufferedOutputChannel;
- this.byteBufferedOutputChannel.setByteBufferedOutputChannelBroker(this);
- }
-
- public void setForwardingChain(final OutputChannelForwardingChain forwardingChain) {
- this.forwardingChain = forwardingChain;
- }
-
-
- @Override
- public boolean hasDataLeft() throws IOException, InterruptedException {
-
- if (this.closeAcknowledgmentReceived) {
- return getNext().hasDataLeft();
- }
-
- if ((this.lastSequenceNumberWithReceiverNotFound + 1) == this.sequenceNumber) {
- return getNext().hasDataLeft();
- }
-
- return true;
- }
-
-
- @Override
- public void processEvent(final AbstractEvent event) {
-
- if (event instanceof ByteBufferedChannelCloseEvent) {
- this.closeAcknowledgmentReceived = true;
- } else if (event instanceof ReceiverNotFoundEvent) {
- this.lastSequenceNumberWithReceiverNotFound = ((ReceiverNotFoundEvent) event).getSequenceNumber();
- } else if (event instanceof AbstractTaskEvent) {
- throw new IllegalStateException("Received synchronous task event " + event);
- }
-
- getNext().processEvent(event);
- }
-
-
- @Override
- public Buffer requestEmptyWriteBuffer() throws InterruptedException, IOException {
-
- if (this.outgoingTransferEnvelope == null) {
- this.outgoingTransferEnvelope = createNewOutgoingTransferEnvelope();
- }
-
- final int uncompressedBufferSize = calculateBufferSize();
-
- return this.outputGateContext.requestEmptyBufferBlocking(uncompressedBufferSize);
- }
-
- /**
- * Creates a new {@link TransferEnvelope} object. The method assigns
- * and increases the sequence number. Moreover, it will look up the list of receivers for this transfer envelope.
- * This method will block until the lookup is completed.
- *
- * @return a new {@link TransferEnvelope} object containing the correct sequence number and receiver list
- */
- private TransferEnvelope createNewOutgoingTransferEnvelope() {
-
- final TransferEnvelope transferEnvelope = new TransferEnvelope(this.sequenceNumber++,
- this.byteBufferedOutputChannel.getJobID(),
- this.byteBufferedOutputChannel.getID());
-
- return transferEnvelope;
- }
-
- /**
- * Calculates the recommended size of the next buffer to be
- * handed to the attached channel object in bytes.
- *
- * @return the recommended size of the next buffer in bytes
- */
- private int calculateBufferSize() {
-
- // TODO: Include latency considerations
- return this.outputGateContext.getMaximumBufferSize();
- }
-
-
- @Override
- public void releaseWriteBuffer(final Buffer buffer) throws IOException, InterruptedException {
-
- // Check for events
- this.forwardingChain.processQueuedEvents();
-
- if (this.outgoingTransferEnvelope == null) {
- throw new IllegalStateException("Cannot find transfer envelope for channel with ID "
- + this.byteBufferedOutputChannel.getID());
- }
-
- // Consistency check
- if (this.outgoingTransferEnvelope.getBuffer() != null) {
- throw new IllegalStateException("Channel " + this.byteBufferedOutputChannel.getID()
- + " has already a buffer attached");
- }
- buffer.flip();
- this.outgoingTransferEnvelope.setBuffer(buffer);
-
- this.forwardingChain.pushEnvelope(this.outgoingTransferEnvelope);
- this.outgoingTransferEnvelope = null;
- }
-
-
- @Override
- public boolean hasDataLeftToTransmit() throws IOException, InterruptedException {
-
- // Check for events
- this.forwardingChain.processQueuedEvents();
-
- return this.forwardingChain.anyForwarderHasDataLeft();
- }
-
-
- @Override
- public void transferEventToInputChannel(final AbstractEvent event) throws IOException, InterruptedException {
-
- if (this.outgoingTransferEnvelope != null) {
- this.outgoingTransferEnvelope.addEvent(event);
- } else {
-
- final TransferEnvelope ephemeralTransferEnvelope = createNewOutgoingTransferEnvelope();
- ephemeralTransferEnvelope.addEvent(event);
-
- this.forwardingChain.pushEnvelope(ephemeralTransferEnvelope);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java
deleted file mode 100644
index 01d8ba2..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputChannelContext.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/***********************************************************************************************************************
- * 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 eu.stratosphere.nephele.event.task.AbstractEvent;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.ChannelType;
-import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelContext;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain;
-
-public final class RuntimeOutputChannelContext extends AbstractOutputChannelContext {
-
- private final AbstractByteBufferedOutputChannel<?> byteBufferedOutputChannel;
-
- RuntimeOutputChannelContext(final AbstractByteBufferedOutputChannel<?> byteBufferedOutputChannel,
- final OutputChannelForwardingChain forwardingChain) {
- super(forwardingChain);
-
- this.byteBufferedOutputChannel = byteBufferedOutputChannel;
- }
-
-
- @Override
- public boolean isInputChannel() {
-
- return false;
- }
-
-
- @Override
- public ChannelID getChannelID() {
-
- return this.byteBufferedOutputChannel.getID();
- }
-
-
- @Override
- public ChannelID getConnectedChannelID() {
-
- return this.byteBufferedOutputChannel.getConnectedChannelID();
- }
-
-
- @Override
- public JobID getJobID() {
-
- return this.byteBufferedOutputChannel.getJobID();
- }
-
-
- @Override
- public ChannelType getType() {
-
- return this.byteBufferedOutputChannel.getType();
- }
-
-
- @Override
- protected void processEventAsynchronously(final AbstractEvent event) {
-
- this.byteBufferedOutputChannel.processEvent(event);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java
deleted file mode 100644
index 72e5047..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeOutputGateContext.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.io.AbstractID;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.channels.AbstractOutputChannel;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.bytebuffered.AbstractByteBufferedOutputChannel;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferAvailabilityListener;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.AbstractOutputChannelForwarder;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelContext;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputChannelForwardingChain;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext;
-
-final class RuntimeOutputGateContext implements BufferProvider, OutputGateContext {
-
- private final RuntimeTaskContext taskContext;
-
- private final OutputGate<? extends IOReadableWritable> outputGate;
-
- RuntimeOutputGateContext(final RuntimeTaskContext taskContext, final OutputGate<? extends IOReadableWritable> outputGate) {
-
- this.taskContext = taskContext;
- this.outputGate = outputGate;
- }
-
- AbstractID getFileOwnerID() {
-
- return this.taskContext.getFileOwnerID();
- }
-
-
- @Override
- public int getMaximumBufferSize() {
-
- return this.taskContext.getMaximumBufferSize();
- }
-
-
- @Override
- public Buffer requestEmptyBuffer(int minimumSizeOfBuffer) throws IOException {
-
- return this.taskContext.requestEmptyBuffer(minimumSizeOfBuffer);
- }
-
-
- @Override
- public Buffer requestEmptyBufferBlocking(int minimumSizeOfBuffer) throws IOException, InterruptedException {
-
- Buffer buffer = this.taskContext.requestEmptyBuffer(minimumSizeOfBuffer);
-
- // No memory-based buffer available
- if (buffer == null) {
- // Wait until a memory-based buffer is available
- buffer = this.taskContext.requestEmptyBufferBlocking(minimumSizeOfBuffer);
- }
-
- return buffer;
- }
-
-
- @Override
- public boolean isShared() {
-
- return this.taskContext.isShared();
- }
-
-
- @Override
- public void reportAsynchronousEvent() {
-
- this.taskContext.reportAsynchronousEvent();
- }
-
-
- @Override
- public GateID getGateID() {
-
- return this.outputGate.getGateID();
- }
-
-
- @Override
- public OutputChannelContext createOutputChannelContext(ChannelID channelID, OutputChannelContext previousContext,
- boolean isReceiverRunning, boolean mergeSpillBuffers) {
-
- if (previousContext != null) {
- throw new IllegalStateException("Found previous output context for channel " + channelID);
- }
-
- AbstractOutputChannel<? extends IOReadableWritable> channel = null;
- for (int i = 0; i < this.outputGate.getNumberOfOutputChannels(); ++i) {
- AbstractOutputChannel<? extends IOReadableWritable> candidateChannel = this.outputGate.getOutputChannel(i);
- if (candidateChannel.getID().equals(channelID)) {
- channel = candidateChannel;
- break;
- }
- }
-
- if (channel == null) {
- throw new IllegalArgumentException("Cannot find output channel with ID " + channelID);
- }
-
- if (!(channel instanceof AbstractByteBufferedOutputChannel)) {
- throw new IllegalStateException("Channel with ID" + channelID
- + " is not of type AbstractByteBufferedOutputChannel");
- }
-
- // The output channel for this context
- final AbstractByteBufferedOutputChannel<? extends IOReadableWritable> outputChannel = (AbstractByteBufferedOutputChannel<? extends IOReadableWritable>) channel;
-
- // Construct the forwarding chain
- RuntimeOutputChannelBroker outputChannelBroker;
- AbstractOutputChannelForwarder last;
- // Construction for in-memory and network channels
- final RuntimeDispatcher runtimeDispatcher = new RuntimeDispatcher(
- this.taskContext.getTransferEnvelopeDispatcher());
- /*
- * final SpillingBarrier spillingBarrier = new SpillingBarrier(isReceiverRunning, mergeSpillBuffers,
- * runtimeDispatcher);
- * final ForwardingBarrier forwardingBarrier = new ForwardingBarrier(channelID, spillingBarrier);
- */
- final ForwardingBarrier forwardingBarrier = new ForwardingBarrier(channelID, runtimeDispatcher);
- outputChannelBroker = new RuntimeOutputChannelBroker(this, outputChannel, forwardingBarrier);
- last = runtimeDispatcher;
-
- final OutputChannelForwardingChain forwardingChain = new OutputChannelForwardingChain(outputChannelBroker, last);
-
- // Set forwarding chain for broker
- outputChannelBroker.setForwardingChain(forwardingChain);
-
- return new RuntimeOutputChannelContext(outputChannel, forwardingChain);
- }
-
-
- @Override
- public boolean registerBufferAvailabilityListener(final BufferAvailabilityListener bufferAvailabilityListener) {
-
- return this.taskContext.registerBufferAvailabilityListener(bufferAvailabilityListener);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java
deleted file mode 100644
index 5a44530..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTask.java
+++ /dev/null
@@ -1,346 +0,0 @@
-/***********************************************************************************************************************
- * 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.Iterator;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.nephele.execution.Environment;
-import eu.stratosphere.nephele.execution.ExecutionListener;
-import eu.stratosphere.nephele.execution.ExecutionObserver;
-import eu.stratosphere.nephele.execution.ExecutionState;
-import eu.stratosphere.nephele.execution.ExecutionStateTransition;
-import eu.stratosphere.nephele.execution.RuntimeEnvironment;
-import eu.stratosphere.nephele.executiongraph.ExecutionVertexID;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.profiling.TaskManagerProfiler;
-import eu.stratosphere.nephele.services.memorymanager.MemoryManager;
-import eu.stratosphere.nephele.taskmanager.Task;
-import eu.stratosphere.nephele.taskmanager.TaskManager;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPoolOwner;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher;
-import eu.stratosphere.nephele.template.AbstractInvokable;
-import eu.stratosphere.util.StringUtils;
-
-public final class RuntimeTask implements Task, ExecutionObserver {
-
- /**
- * The log object used for debugging.
- */
- private static final Log LOG = LogFactory.getLog(RuntimeTask.class);
-
- private final ExecutionVertexID vertexID;
-
- private final RuntimeEnvironment environment;
-
- private final TaskManager taskManager;
-
- /**
- * Stores whether the task has been canceled.
- */
- private volatile boolean isCanceled = false;
-
- /**
- * The current execution state of the task
- */
- private volatile ExecutionState executionState = ExecutionState.STARTING;
-
- private Queue<ExecutionListener> registeredListeners = new ConcurrentLinkedQueue<ExecutionListener>();
-
- public RuntimeTask(final ExecutionVertexID vertexID, final RuntimeEnvironment environment,
- final TaskManager taskManager) {
-
- this.vertexID = vertexID;
- this.environment = environment;
- this.taskManager = taskManager;
-
- this.environment.setExecutionObserver(this);
- }
-
-
- @Override
- public void executionStateChanged(final ExecutionState newExecutionState, final String optionalMessage) {
-
- // Check the state transition
- ExecutionStateTransition.checkTransition(false, getTaskName(), this.executionState, newExecutionState);
-
- // Make sure the reason for a transition to FAILED appears in the log files
- if (newExecutionState == ExecutionState.FAILED) {
- LOG.error(optionalMessage);
- }
-
- // Notify all listener objects
- final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
- while (it.hasNext()) {
- it.next().executionStateChanged(this.environment.getJobID(), this.vertexID, newExecutionState,
- optionalMessage);
- }
-
- // Store the new execution state
- this.executionState = newExecutionState;
-
- // Finally propagate the state change to the job manager
- this.taskManager.executionStateChanged(this.environment.getJobID(), this.vertexID, newExecutionState,
- optionalMessage);
- }
-
- /**
- * Returns the name of the task associated with this observer object.
- *
- * @return the name of the task associated with this observer object
- */
- private String getTaskName() {
-
- return this.environment.getTaskName() + " (" + (this.environment.getIndexInSubtaskGroup() + 1) + "/"
- + this.environment.getCurrentNumberOfSubtasks() + ")";
- }
-
-
- @Override
- public void userThreadStarted(final Thread userThread) {
-
- // Notify the listeners
- final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
- while (it.hasNext()) {
- it.next().userThreadStarted(this.environment.getJobID(), this.vertexID, userThread);
- }
- }
-
-
- @Override
- public void userThreadFinished(final Thread userThread) {
-
- // Notify the listeners
- final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
- while (it.hasNext()) {
- it.next().userThreadFinished(this.environment.getJobID(), this.vertexID, userThread);
- }
- }
-
- /**
- * Registers the {@link ExecutionListener} object for this task. This object
- * will be notified about important events during the task execution.
- *
- * @param executionListener
- * the object to be notified for important events during the task execution
- */
-
- public void registerExecutionListener(final ExecutionListener executionListener) {
-
- this.registeredListeners.add(executionListener);
- }
-
- /**
- * Unregisters the {@link ExecutionListener} object for this environment. This object
- * will no longer be notified about important events during the task execution.
- *
- * @param executionListener
- * the lister object to be unregistered
- */
-
- public void unregisterExecutionListener(final ExecutionListener executionListener) {
-
- this.registeredListeners.remove(executionListener);
- }
-
-
- @Override
- public void markAsFailed() {
-
- executionStateChanged(ExecutionState.FAILED, "Execution thread died unexpectedly");
- }
-
-
- @Override
- public void cancelExecution() {
-
- cancelOrKillExecution(true);
- }
-
-
- @Override
- public void killExecution() {
-
- cancelOrKillExecution(false);
- }
-
- /**
- * Cancels or kills the task.
- *
- * @param cancel
- * <code>true/code> if the task shall be canceled, <code>false</code> if it shall be killed
- */
- private void cancelOrKillExecution(final boolean cancel) {
-
- final Thread executingThread = this.environment.getExecutingThread();
-
- if (executingThread == null) {
- return;
- }
-
- if (this.executionState != ExecutionState.RUNNING && this.executionState != ExecutionState.FINISHING) {
- return;
- }
-
- LOG.info((cancel ? "Canceling " : "Killing ") + this.environment.getTaskNameWithIndex());
-
- if (cancel) {
- this.isCanceled = true;
- // Change state
- executionStateChanged(ExecutionState.CANCELING, null);
-
- // Request user code to shut down
- try {
- final AbstractInvokable invokable = this.environment.getInvokable();
- if (invokable != null) {
- invokable.cancel();
- }
- } catch (Throwable e) {
- LOG.error(StringUtils.stringifyException(e));
- }
- }
-
- // Continuously interrupt the user thread until it changed to state CANCELED
- while (true) {
-
- executingThread.interrupt();
-
- if (!executingThread.isAlive()) {
- break;
- }
-
- try {
- executingThread.join(1000);
- } catch (InterruptedException e) {}
-
- if (!executingThread.isAlive()) {
- break;
- }
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Sending repeated " + (cancel == true ? "canceling" : "killing") + " signal to " +
- this.environment.getTaskName() + " with state " + this.executionState);
- }
- }
- }
-
-
- @Override
- public void startExecution() {
-
- final Thread thread = this.environment.getExecutingThread();
- thread.start();
- }
-
-
- @Override
- public boolean isCanceled() {
-
- return this.isCanceled;
- }
-
-
- @Override
- public boolean isTerminated() {
-
- final Thread executingThread = this.environment.getExecutingThread();
- if (executingThread.getState() == Thread.State.TERMINATED) {
- return true;
- }
-
- return false;
- }
-
-
- @Override
- public Environment getEnvironment() {
-
- return this.environment;
- }
-
- /**
- * Returns the runtime environment associated with this task.
- *
- * @return the runtime environment associated with this task
- */
- public RuntimeEnvironment getRuntimeEnvironment() {
-
- return this.environment;
- }
-
-
- @Override
- public JobID getJobID() {
-
- return this.environment.getJobID();
- }
-
-
- @Override
- public ExecutionVertexID getVertexID() {
-
- return this.vertexID;
- }
-
-
- @Override
- public void registerProfiler(final TaskManagerProfiler taskManagerProfiler, final Configuration jobConfiguration) {
-
- taskManagerProfiler.registerExecutionListener(this, jobConfiguration);
- }
-
-
- @Override
- public void unregisterMemoryManager(final MemoryManager memoryManager) {
-
- if (memoryManager != null) {
- memoryManager.releaseAll(this.environment.getInvokable());
- }
- }
-
-
- @Override
- public void unregisterProfiler(final TaskManagerProfiler taskManagerProfiler) {
-
- if (taskManagerProfiler != null) {
- taskManagerProfiler.unregisterExecutionListener(this.vertexID);
- }
- }
-
-
- @Override
- public TaskContext createTaskContext(final TransferEnvelopeDispatcher transferEnvelopeDispatcher,
- final LocalBufferPoolOwner previousBufferPoolOwner) {
-
- if (previousBufferPoolOwner != null) {
- throw new IllegalStateException("Vertex " + this.vertexID + " has a previous buffer pool owner");
- }
-
- return new RuntimeTaskContext(this, transferEnvelopeDispatcher);
- }
-
-
- @Override
- public ExecutionState getExecutionState() {
-
- return this.executionState;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java
deleted file mode 100644
index 7f3a9a0..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/runtime/RuntimeTaskContext.java
+++ /dev/null
@@ -1,211 +0,0 @@
-/***********************************************************************************************************************
- * 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.io.IOException;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.execution.RuntimeEnvironment;
-import eu.stratosphere.nephele.io.AbstractID;
-import eu.stratosphere.nephele.io.GateID;
-import eu.stratosphere.nephele.io.InputGate;
-import eu.stratosphere.nephele.io.OutputGate;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferAvailabilityListener;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.LocalBufferPool;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.InputGateContext;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.OutputGateContext;
-import eu.stratosphere.nephele.taskmanager.bytebuffered.TaskContext;
-import eu.stratosphere.nephele.taskmanager.transferenvelope.TransferEnvelopeDispatcher;
-
-public final class RuntimeTaskContext implements BufferProvider, TaskContext {
-
- private final LocalBufferPool localBufferPool;
-
- private final RuntimeTask task;
-
- private final int numberOfOutputChannels;
-
- private final TransferEnvelopeDispatcher transferEnvelopeDispatcher;
-
- RuntimeTaskContext(final RuntimeTask task, final TransferEnvelopeDispatcher transferEnvelopeDispatcher) {
-
- this.localBufferPool = new LocalBufferPool(1, false);
- this.task = task;
-
- final RuntimeEnvironment environment = task.getRuntimeEnvironment();
-
- // Compute number of output input channels
- int nooc = 0;
- for (int i = 0; i < environment.getNumberOfOutputGates(); ++i) {
- final OutputGate<? extends IOReadableWritable> outputGate = environment.getOutputGate(i);
- if (outputGate.isBroadcast()) {
- ++nooc;
- } else {
- nooc += outputGate.getNumberOfOutputChannels();
- }
- }
- this.numberOfOutputChannels = nooc;
-
- this.transferEnvelopeDispatcher = transferEnvelopeDispatcher;
- }
-
- TransferEnvelopeDispatcher getTransferEnvelopeDispatcher() {
-
- return this.transferEnvelopeDispatcher;
- }
-
-
-
- @Override
- public Buffer requestEmptyBuffer(final int minimumSizeOfBuffer) throws IOException {
-
- return this.localBufferPool.requestEmptyBuffer(minimumSizeOfBuffer);
- }
-
-
- @Override
- public Buffer requestEmptyBufferBlocking(int minimumSizeOfBuffer) throws IOException,
- InterruptedException {
-
- return this.localBufferPool.requestEmptyBufferBlocking(minimumSizeOfBuffer);
- }
-
-
- @Override
- public int getMaximumBufferSize() {
-
- return this.localBufferPool.getMaximumBufferSize();
- }
-
-
- @Override
- public void clearLocalBufferPool() {
-
- // Clear the buffer cache
- this.localBufferPool.destroy();
-
- }
-
-
- @Override
- public boolean isShared() {
-
- return false;
- }
-
-
- @Override
- public void logBufferUtilization() {
-
- final int ava = this.localBufferPool.getNumberOfAvailableBuffers();
- final int req = this.localBufferPool.getRequestedNumberOfBuffers();
- final int des = this.localBufferPool.getDesignatedNumberOfBuffers();
-
- final RuntimeEnvironment environment = this.task.getRuntimeEnvironment();
-
- System.out.println("\t\t" + environment.getTaskNameWithIndex() + ": " + ava + " available, " + req
- + " requested, " + des + " designated");
- }
-
-
-
- @Override
- public void reportAsynchronousEvent() {
-
- this.localBufferPool.reportAsynchronousEvent();
- }
-
-
- @Override
- public int getNumberOfChannels() {
-
- return this.numberOfOutputChannels;
- }
-
-
- @Override
- public void setDesignatedNumberOfBuffers(int numberOfBuffers) {
-
- this.localBufferPool.setDesignatedNumberOfBuffers(numberOfBuffers);
- }
-
- AbstractID getFileOwnerID() {
-
- return this.task.getVertexID();
- }
-
-
- @Override
- public OutputGateContext createOutputGateContext(final GateID gateID) {
-
- if (gateID == null) {
- throw new IllegalArgumentException("Argument gateID must not be null");
- }
-
- OutputGate<? extends IOReadableWritable> outputGate = null;
- final RuntimeEnvironment re = this.task.getRuntimeEnvironment();
- for (int i = 0; i < re.getNumberOfOutputGates(); ++i) {
- final OutputGate<? extends IOReadableWritable> candidateGate = re.getOutputGate(i);
- if (candidateGate.getGateID().equals(gateID)) {
- outputGate = candidateGate;
- break;
- }
- }
-
- if (outputGate == null) {
- throw new IllegalStateException("Cannot find output gate with ID " + gateID);
- }
-
- return new RuntimeOutputGateContext(this, outputGate);
- }
-
-
- @Override
- public InputGateContext createInputGateContext(final GateID gateID) {
-
- if (gateID == null) {
- throw new IllegalArgumentException("Argument gateID must not be null");
- }
-
- InputGate<? extends IOReadableWritable> inputGate = null;
- final RuntimeEnvironment re = this.task.getRuntimeEnvironment();
- for (int i = 0; i < re.getNumberOfInputGates(); ++i) {
- final InputGate<? extends IOReadableWritable> candidateGate = re.getInputGate(i);
- if (candidateGate.getGateID().equals(gateID)) {
- inputGate = candidateGate;
- break;
- }
- }
-
- if (inputGate == null) {
- throw new IllegalStateException("Cannot find input gate with ID " + gateID);
- }
-
- return new RuntimeInputGateContext(re.getTaskNameWithIndex(), this.transferEnvelopeDispatcher, inputGate);
- }
-
- public LocalBufferPool getLocalBufferPool() {
-
- return this.localBufferPool;
- }
-
-
- @Override
- public boolean registerBufferAvailabilityListener(final BufferAvailabilityListener bufferAvailabilityListener) {
-
- return this.localBufferPool.registerBufferAvailabilityListener(bufferAvailabilityListener);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractDeserializer.java
deleted file mode 100644
index 3a810f6..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractDeserializer.java
+++ /dev/null
@@ -1,355 +0,0 @@
-/***********************************************************************************************************************
- * 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.transferenvelope;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.ReadableByteChannel;
-
-import eu.stratosphere.nephele.event.task.EventList;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.io.channels.DefaultDeserializer;
-import eu.stratosphere.nephele.jobgraph.JobID;
-
-public abstract class AbstractDeserializer {
-
- private enum DeserializationState {
- NOTDESERIALIZED,
- SEQUENCENUMBERDESERIALIZED,
- JOBIDDESERIALIZED,
- SOURCEDESERIALIZED,
- NOTIFICATIONSDESERIALIZED,
- FULLYDESERIALIZED
- };
-
- private static final int SIZEOFINT = 4;
-
- private TransferEnvelope transferEnvelope = null;
-
- private DeserializationState deserializationState = DeserializationState.NOTDESERIALIZED;
-
- private final DefaultDeserializer<ChannelID> channelIDDeserializationBuffer = new DefaultDeserializer<ChannelID>(
- ChannelID.class, true);
-
- private final DefaultDeserializer<JobID> jobIDDeserializationBuffer = new DefaultDeserializer<JobID>(
- JobID.class, true);
-
- private final DefaultDeserializer<EventList> notificationListDeserializationBuffer = new DefaultDeserializer<EventList>(
- EventList.class, true);
-
- private final ByteBuffer tempBuffer = ByteBuffer.allocate(8); // TODO: Make this configurable
-
- private boolean bufferExistanceDeserialized = false;
-
- private boolean eventListExistanceDeserialized = false;
-
- private boolean sequenceNumberDeserializationStarted = false;
-
- private int sizeOfBuffer = -1;
-
- private int deserializedSequenceNumber = -1;
-
- private Buffer buffer = null;
-
- private JobID deserializedJobID = null;
-
- private ChannelID deserializedSourceID = null;
-
- private EventList deserializedEventList = null;
-
- public void read(ReadableByteChannel readableByteChannel) throws IOException, NoBufferAvailableException {
-
- while (true) {
-
- // System.out.println("INCOMING State: " + this.deserializationState);
-
- boolean waitingForMoreData = false;
-
- switch (deserializationState) {
- case NOTDESERIALIZED:
- waitingForMoreData = readSequenceNumber(readableByteChannel);
- break;
- case SEQUENCENUMBERDESERIALIZED:
- waitingForMoreData = readID(readableByteChannel);
- break;
- case JOBIDDESERIALIZED:
- waitingForMoreData = readID(readableByteChannel);
- break;
- case SOURCEDESERIALIZED:
- waitingForMoreData = readNotificationList(readableByteChannel);
- break;
- case NOTIFICATIONSDESERIALIZED:
- waitingForMoreData = readBuffer(readableByteChannel);
- break;
- case FULLYDESERIALIZED:
- return;
- }
-
- if (waitingForMoreData) {
- return;
- }
-
- }
- }
-
- protected final ByteBuffer getTempBuffer() {
- return this.tempBuffer;
- }
-
- protected void setBuffer(final Buffer buffer) {
- this.buffer = buffer;
- }
-
- protected int getSizeOfBuffer() {
- return this.sizeOfBuffer;
- }
-
- protected JobID getDeserializedJobID() {
- return this.deserializedJobID;
- }
-
- protected ChannelID getDeserializedSourceID() {
- return this.deserializedSourceID;
- }
-
- private boolean readSequenceNumber(ReadableByteChannel readableByteChannel) throws IOException {
-
- if (!this.sequenceNumberDeserializationStarted) {
- this.tempBuffer.position(0);
- this.tempBuffer.limit(SIZEOFINT);
- this.sequenceNumberDeserializationStarted = true;
- }
-
- if (readableByteChannel.read(this.tempBuffer) == -1) {
-
- if (this.tempBuffer.position() == 0) {
- // Regular end of stream
- throw new EOFException();
- } else {
- throw new IOException("Unexpected end of stream while deserializing the sequence number");
- }
- }
-
- if (!this.tempBuffer.hasRemaining()) {
-
- this.deserializedSequenceNumber = byteBufferToInteger(this.tempBuffer, 0);
- if (this.deserializedSequenceNumber < 0) {
- throw new IOException("Received invalid sequence number: " + this.deserializedSequenceNumber);
- }
-
- this.deserializationState = DeserializationState.SEQUENCENUMBERDESERIALIZED;
- this.sequenceNumberDeserializationStarted = false;
- this.transferEnvelope = null;
- this.sizeOfBuffer = -1;
- this.bufferExistanceDeserialized = false;
- this.eventListExistanceDeserialized = false;
- this.tempBuffer.clear();
- this.buffer = null;
- this.jobIDDeserializationBuffer.clear();
- this.channelIDDeserializationBuffer.clear();
- this.deserializedEventList = null;
- return false;
- }
-
- return true;
- }
-
- private boolean readID(ReadableByteChannel readableByteChannel) throws IOException {
-
- if (this.deserializationState == DeserializationState.SEQUENCENUMBERDESERIALIZED) {
-
- this.deserializedJobID = this.jobIDDeserializationBuffer.readData(null, readableByteChannel);
- if (this.deserializedJobID == null) {
- return true;
- }
-
- this.deserializationState = DeserializationState.JOBIDDESERIALIZED;
-
- } else {
-
- this.deserializedSourceID = this.channelIDDeserializationBuffer.readData(null, readableByteChannel);
- if (this.deserializedSourceID == null) {
- return true;
- }
-
- this.deserializationState = DeserializationState.SOURCEDESERIALIZED;
- }
-
- return false;
- }
-
- private boolean readNotificationList(ReadableByteChannel readableByteChannel) throws IOException {
-
- if (!this.eventListExistanceDeserialized) {
- this.tempBuffer.position(0);
- this.tempBuffer.limit(1);
- readableByteChannel.read(this.tempBuffer);
-
- if (this.tempBuffer.hasRemaining()) {
- return true;
- }
-
- this.eventListExistanceDeserialized = true;
- final boolean eventListFollows = (this.tempBuffer.get(0) == (byte) 1);
- this.tempBuffer.clear();
-
- if (!eventListFollows) {
- // No event list here
- this.transferEnvelope = new TransferEnvelope(this.deserializedSequenceNumber, this.deserializedJobID,
- this.deserializedSourceID, this.deserializedEventList);
- this.deserializationState = DeserializationState.NOTIFICATIONSDESERIALIZED;
- return false;
- }
- }
-
- this.deserializedEventList = this.notificationListDeserializationBuffer.readData(null, readableByteChannel);
- if (this.deserializedEventList == null) {
- return true;
- } else {
- this.transferEnvelope = new TransferEnvelope(this.deserializedSequenceNumber, this.deserializedJobID,
- this.deserializedSourceID, this.deserializedEventList);
- this.deserializationState = DeserializationState.NOTIFICATIONSDESERIALIZED;
- return false;
- }
- }
-
- /**
- * Read the buffer's actual data from the stream.
- *
- * @param readableByteChannel
- * the stream to read the buffer data from
- * @return <code>true</code> if more buffer data need to be read from the stream, <code>false</code> otherwise
- * @throws IOException
- * thrown if an I/O error occurred while reading data from the stream
- * @throws NoBufferAvailableException
- * thrown if the deserialization process could not be continued due to a lack of buffers
- */
- protected abstract boolean readBufferData(ReadableByteChannel readableByteChannel) throws IOException,
- NoBufferAvailableException;
-
- private boolean readBuffer(final ReadableByteChannel readableByteChannel) throws IOException,
- NoBufferAvailableException {
-
- if (!this.bufferExistanceDeserialized) {
-
- this.tempBuffer.position(0);
- this.tempBuffer.limit(1);
-
- final int bytesRead = readableByteChannel.read(this.tempBuffer);
- if (bytesRead == -1) {
- if (this.tempBuffer.get(0) == 0 && this.tempBuffer.position() == 1) { // Regular end, no
- // buffer will follow
- throw new EOFException();
- } else {
- throw new IOException("Deserialization error: Expected at least "
- + this.tempBuffer.remaining() + " more bytes to follow");
- }
- } else if (bytesRead == 0) {
- try {
- Thread.sleep(50);
- } catch (InterruptedException e) {
- }
- }
-
- if (!this.tempBuffer.hasRemaining()) {
- this.bufferExistanceDeserialized = true;
- this.tempBuffer.position(0);
- this.tempBuffer.limit(SIZEOFINT);
- if (this.tempBuffer.get(0) == 0) {
- // No buffer will follow, we are done
- this.transferEnvelope.setBuffer(null);
- this.deserializationState = DeserializationState.FULLYDESERIALIZED;
- return false;
- }
- } else {
- return true;
- }
- }
-
- if (this.sizeOfBuffer < 0) {
-
- // We need to deserialize the size of the buffer
- final int bytesRead = readableByteChannel.read(this.tempBuffer);
- if (bytesRead == -1) {
- throw new IOException("Deserialization error: Expected at least " + this.tempBuffer.remaining()
- + " more bytes to follow");
- }
-
- if (!this.tempBuffer.hasRemaining()) {
- this.sizeOfBuffer = byteBufferToInteger(this.tempBuffer, 0);
- // System.out.println("INCOMING: Buffer size is " + this.sizeOfBuffer);
-
- if (this.sizeOfBuffer <= 0) {
- throw new IOException("Invalid buffer size: " + this.sizeOfBuffer);
- }
- } else {
- return true;
- }
- }
-
- if (readBufferData(readableByteChannel)) {
- return true;
- }
-
- this.transferEnvelope.setBuffer(this.buffer);
- this.deserializationState = DeserializationState.FULLYDESERIALIZED;
- return false;
- }
-
- public TransferEnvelope getFullyDeserializedTransferEnvelope() {
-
- if (this.deserializationState == DeserializationState.FULLYDESERIALIZED) {
- this.deserializationState = DeserializationState.NOTDESERIALIZED;
- return this.transferEnvelope;
- }
-
- return null;
- }
-
- public Buffer getBuffer() {
- return this.buffer;
- }
-
- public void reset() {
- this.deserializationState = DeserializationState.NOTDESERIALIZED;
- this.sequenceNumberDeserializationStarted = false;
- }
-
- public boolean hasUnfinishedData() {
-
- if (this.deserializationState != DeserializationState.NOTDESERIALIZED) {
- return true;
- }
-
- return this.channelIDDeserializationBuffer.hasUnfinishedData();
- }
-
- private int byteBufferToInteger(ByteBuffer byteBuffer, int offset) throws IOException {
-
- int integer = 0;
-
- if ((offset + SIZEOFINT) > byteBuffer.limit()) {
- throw new IOException("Cannot convert byte buffer to integer, not enough data in byte buffer ("
- + byteBuffer.limit() + ")");
- }
-
- for (int i = 0; i < SIZEOFINT; ++i) {
- integer |= (byteBuffer.get((offset + SIZEOFINT - 1) - i) & 0xff) << (i << 3);
- }
-
- return integer;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractSerializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractSerializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractSerializer.java
deleted file mode 100644
index bdbc592..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/AbstractSerializer.java
+++ /dev/null
@@ -1,274 +0,0 @@
-/***********************************************************************************************************************
- * 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.transferenvelope;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.WritableByteChannel;
-
-import eu.stratosphere.core.io.IOReadableWritable;
-import eu.stratosphere.nephele.event.task.EventList;
-import eu.stratosphere.nephele.io.AbstractID;
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.SerializationBuffer;
-
-public abstract class AbstractSerializer {
-
- private enum SerializationState {
- NOTSERIALIZED,
- SEQUENCENUMBERSERIALIZED,
- JOBIDSERIALIZED,
- SOURCESERIALIZED,
- NOTIFICATIONSSERIALIZED,
- FULLYSERIALIZED
- };
-
- private final static int SIZEOFINT = 4;
-
- private TransferEnvelope transferEnvelope = null;
-
- private SerializationState serializationState;
-
- private final SerializationBuffer<IOReadableWritable> serializationBuffer = new SerializationBuffer<IOReadableWritable>();
-
- private final ByteBuffer tempBuffer = ByteBuffer.allocate(64); // TODO: Make this configurable
-
- private boolean serializationStarted = false;
-
- private boolean bufferExistanceSerialized = false;
-
- private boolean eventListExistanceSerialized = false;
-
- public final void setTransferEnvelope(TransferEnvelope transferEnvelope) {
-
- this.transferEnvelope = transferEnvelope;
- reset();
- }
-
- protected final SerializationBuffer<IOReadableWritable> getSerializationBuffer() {
-
- return this.serializationBuffer;
- }
-
- protected final ByteBuffer getTempBuffer() {
- return this.tempBuffer;
- }
-
- public final boolean write(WritableByteChannel writableByteChannel) throws IOException {
-
- while (true) {
-
- boolean moreDataFollows = false;
-
- // System.out.println("OUTGOING State: " + this.serializationState);
-
- switch (serializationState) {
- case NOTSERIALIZED:
- moreDataFollows = writeSequenceNumber(writableByteChannel, this.transferEnvelope.getSequenceNumber());
- break;
- case SEQUENCENUMBERSERIALIZED:
- moreDataFollows = writeID(writableByteChannel, this.transferEnvelope.getJobID());
- break;
- case JOBIDSERIALIZED:
- moreDataFollows = writeID(writableByteChannel, this.transferEnvelope.getSource());
- break;
- case SOURCESERIALIZED:
- moreDataFollows = writeNotification(writableByteChannel, this.transferEnvelope.getEventList());
- break;
- case NOTIFICATIONSSERIALIZED:
- moreDataFollows = writeBuffer(writableByteChannel, this.transferEnvelope.getBuffer());
- break;
- case FULLYSERIALIZED:
- return false;
- }
-
- if (moreDataFollows) {
- return true;
- }
- }
- }
-
- private boolean writeSequenceNumber(WritableByteChannel writableByteChannel, int sequenceNumber) throws IOException {
-
- if (sequenceNumber < 0) {
- throw new IOException("Invalid sequence number: " + sequenceNumber);
- }
-
- if (!this.serializationStarted) {
- this.tempBuffer.clear();
- integerToByteBuffer(sequenceNumber, 0, this.tempBuffer);
- this.serializationStarted = true;
- }
-
- if (writableByteChannel.write(this.tempBuffer) == -1) {
- throw new IOException("Unexpected end of stream while serializing the sequence number");
- }
-
- if (!this.tempBuffer.hasRemaining()) {
- this.serializationState = SerializationState.SEQUENCENUMBERSERIALIZED;
- this.serializationStarted = false;
- return false;
- }
-
- return true;
- }
-
- private boolean writeID(WritableByteChannel writableByteChannel, AbstractID id) throws IOException {
-
- if (!writeIOReadableWritable(writableByteChannel, id)) {
- // We're done, all the data has been written to the channel
- if (this.serializationState == SerializationState.SEQUENCENUMBERSERIALIZED) {
- // System.out.println("OUTGOING Serialized source: " + channelID);
- this.serializationState = SerializationState.JOBIDSERIALIZED;
- } else {
- // System.out.println("OUTGOING Serialized target: " + channelID);
- this.serializationState = SerializationState.SOURCESERIALIZED;
- }
- return false;
- }
-
- return true;
- }
-
- private boolean writeIOReadableWritable(WritableByteChannel writableByteChannel,
- IOReadableWritable ioReadableWritable) throws IOException {
-
- if (!this.serializationStarted) {
- this.serializationBuffer.clear();
- this.serializationBuffer.serialize(ioReadableWritable);
- this.serializationStarted = true;
- }
-
- if (this.serializationBuffer.dataLeftFromPreviousSerialization()) {
- this.serializationBuffer.read(writableByteChannel);
- } else {
- this.serializationStarted = false;
- return false;
- }
-
- return true;
- }
-
- private boolean writeNotification(WritableByteChannel writableByteChannel, EventList notificationList)
- throws IOException {
-
- if (!this.eventListExistanceSerialized) {
- this.tempBuffer.position(0);
- if (notificationList == null) {
- this.tempBuffer.put(0, (byte) 0);
- } else {
- this.tempBuffer.put(0, (byte) 1);
- }
- this.tempBuffer.limit(1);
-
- writableByteChannel.write(this.tempBuffer);
- if (this.tempBuffer.hasRemaining()) {
- return true;
- }
-
- this.eventListExistanceSerialized = true;
- }
-
- if (notificationList != null) {
- if (writeIOReadableWritable(writableByteChannel, notificationList)) {
- return true;
- }
- }
-
- this.serializationState = SerializationState.NOTIFICATIONSSERIALIZED;
- return false;
- }
-
- public void reset() {
- this.serializationState = SerializationState.NOTSERIALIZED;
- this.serializationStarted = false;
- this.bufferExistanceSerialized = false;
- this.eventListExistanceSerialized = false;
- }
-
- private boolean writeBuffer(WritableByteChannel writableByteChannel, Buffer buffer) throws IOException {
-
- while (true) {
-
- if (!this.bufferExistanceSerialized) {
-
- if (!this.serializationStarted) {
- this.tempBuffer.position(0);
-
- if (buffer == null) {
- this.tempBuffer.put(0, (byte) 0);
- this.tempBuffer.limit(1);
- } else {
- this.tempBuffer.put(0, (byte) 1);
- // System.out.println("OUTGOING: Buffer size is " + buffer.size());
- integerToByteBuffer(buffer.size(), 1, this.tempBuffer);
- }
- this.serializationStarted = true;
- }
-
- if (this.tempBuffer.hasRemaining()) {
- if (writableByteChannel.write(tempBuffer) == 0) {
- return true;
- }
- } else {
- this.bufferExistanceSerialized = true;
- this.serializationStarted = false;
- if (buffer == null) {
- // That's it, we're done. No buffer will follow
- this.serializationState = SerializationState.FULLYSERIALIZED;
- return false;
- }
- }
-
- } else {
-
- if (!writeBufferData(writableByteChannel, buffer)) {
- this.serializationState = SerializationState.FULLYSERIALIZED;
- return false;
- }
-
- return true;
- }
- }
- }
-
- /**
- * Writes the buffer's actual data.
- *
- * @param writableByteChannel
- * the channel to write the buffer data to
- * @param buffer
- * the buffer whose data shall be written
- * @return <code>true</code> if the buffer has more data to be written, <code>false</code> otherwise
- * @throws IOException
- * thrown if an I/O error occurs while writing the buffer's data
- */
- protected abstract boolean writeBufferData(WritableByteChannel writableByteChannel, Buffer buffer)
- throws IOException;
-
- private void integerToByteBuffer(int integerToSerialize, int offset, ByteBuffer byteBuffer) throws IOException {
-
- if ((offset + SIZEOFINT) > byteBuffer.capacity()) {
- throw new IOException("Cannot convert integer to byte buffer, buffer is too small (" + byteBuffer.limit()
- + ", required " + (offset + SIZEOFINT) + ")");
- }
-
- byteBuffer.limit(offset + SIZEOFINT);
-
- for (int i = 0; i < SIZEOFINT; ++i) {
- final int shift = i << 3; // i * 8
- byteBuffer.put((offset + SIZEOFINT - 1) - i, (byte) ((integerToSerialize & (0xff << shift)) >>> shift));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CapacityConstrainedArrayQueue.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CapacityConstrainedArrayQueue.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CapacityConstrainedArrayQueue.java
deleted file mode 100644
index 9e9ac66..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/CapacityConstrainedArrayQueue.java
+++ /dev/null
@@ -1,322 +0,0 @@
-/***********************************************************************************************************************
- * 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.transferenvelope;
-
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-import java.util.Queue;
-
-/**
- * This class implements a simple, capacity-constrained queue. The implementation is highlighted by its small
- * overhead in terms of memory consumption and management.
- * <p>
- * This class is not thread-safe.
- *
- * @param <E>
- * the type of the elements stored in this queue
- */
-public final class CapacityConstrainedArrayQueue<E> implements Queue<E> {
-
- /**
- * The maximum capacity supported by this type of queue.
- */
- private static final int MAX_CAPACITY = 128;
-
- /**
- * The array storing the actual elements of the queue.
- */
- private final E[] elements;
-
- /**
- * Index to the current head of the queue.
- */
- private byte head = 0;
-
- /**
- * Index to the current tail of the queue.
- */
- private byte tail = 0;
-
- /**
- * The current size of the queue.
- */
- private byte size = 0;
-
- /**
- * Constructs a new capacity-constrained array queue.
- *
- * @param capacity
- * the capacity limit of the queue
- */
- @SuppressWarnings("unchecked")
- public CapacityConstrainedArrayQueue(final int capacity) {
-
- if (capacity > MAX_CAPACITY) {
- throw new IllegalArgumentException("This queue does only support capacities up to " + MAX_CAPACITY);
- }
-
- this.elements = (E[]) new Object[capacity];
- }
-
-
- @Override
- public boolean addAll(final Collection<? extends E> c) {
-
- throw new UnsupportedOperationException("addAll is not supported on this type of queue");
- }
-
- /**
- * Checks if there is capacity left in the queue.
- *
- * @return <code>true</code> if there is capacity left in the queue, <code>false</code> otherwise
- */
- private boolean capacityLeft() {
-
- return ((this.elements.length - this.size) > 0);
- }
-
- /**
- * Increments the head of the queue.
- */
- private void incrementHead() {
-
- if (++this.head == this.elements.length) {
- this.head = 0;
- }
- }
-
- /**
- * Increments the tail of the queue.
- */
- private void incrementTail() {
-
- if (++this.tail == this.elements.length) {
- this.tail = 0;
- }
- }
-
-
- @Override
- public void clear() {
-
- this.head = 0;
- this.tail = 0;
- this.size = 0;
- }
-
-
- @Override
- public boolean containsAll(final Collection<?> c) {
-
- throw new UnsupportedOperationException("containsAll is not supported on this type of queue");
- }
-
-
- @Override
- public boolean isEmpty() {
-
- return (this.size == 0);
- }
-
-
- @Override
- public boolean removeAll(final Collection<?> c) {
-
- throw new UnsupportedOperationException("removeAll is not supported on this type of queue");
- }
-
-
- @Override
- public boolean retainAll(final Collection<?> c) {
-
- throw new UnsupportedOperationException("retainAll is not supported on this type of queue");
- }
-
-
- @Override
- public Object[] toArray() {
-
- throw new UnsupportedOperationException("toArray is not supported on this type of queue");
- }
-
-
- @Override
- public <T> T[] toArray(T[] a) {
-
- throw new UnsupportedOperationException("toArray is not supported on this type of queue");
- }
-
-
- @Override
- public boolean add(final E arg0) {
-
- throw new UnsupportedOperationException("add is not supported on this type of queue");
- }
-
-
- @Override
- public boolean contains(final Object arg0) {
-
- throw new UnsupportedOperationException("contains is not supported on this type of queue");
- }
-
-
- @Override
- public E element() {
-
- throw new UnsupportedOperationException("element is not supported on this type of queue");
- }
-
-
- @Override
- public Iterator<E> iterator() {
-
- return new CapacityConstrainedArrayQueueIterator(this.head);
- }
-
-
- @Override
- public boolean offer(final E arg0) {
-
- if (!capacityLeft()) {
- return false;
- }
-
- this.elements[this.tail] = arg0;
- incrementTail();
- ++this.size;
-
- return true;
- }
-
-
- @Override
- public E peek() {
-
- if (isEmpty()) {
- return null;
- }
-
- return this.elements[this.head];
- }
-
-
- @Override
- public E poll() {
-
- if (isEmpty()) {
- return null;
- }
-
- final E retVal = this.elements[this.head];
- incrementHead();
- --this.size;
-
- return retVal;
- }
-
-
- @Override
- public E remove() {
-
- final E retVal = poll();
- if (retVal == null) {
- throw new NoSuchElementException();
- }
-
- return retVal;
- }
-
-
- @Override
- public boolean remove(final Object arg0) {
-
- throw new UnsupportedOperationException("remove is not supported on this type of queue");
- }
-
-
- @Override
- public int size() {
-
- return this.size;
- }
-
- /**
- * This class implements an iterator for the capacity-constrained array queue.
- *
- */
- private final class CapacityConstrainedArrayQueueIterator implements Iterator<E> {
-
- /**
- * The current position of the iterator.
- */
- private byte pos;
-
- /**
- * Counter how many this the position index has been modified.
- */
- private byte count = 0;
-
- /**
- * Constructs a new capacity-constrained array queue iterator.
- *
- * @param startPos
- * the start position of the iterator
- */
- private CapacityConstrainedArrayQueueIterator(final byte startPos) {
- this.pos = startPos;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public boolean hasNext() {
-
- if (this.count < size) {
- return true;
- }
-
- return false;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public E next() {
-
- final E retVal = elements[this.pos];
-
- if (++this.pos == elements.length) {
- this.pos = 0;
- }
-
- ++this.count;
-
- return retVal;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public void remove() {
-
- throw new UnsupportedOperationException("remove is not supported by this iterator");
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializer.java
deleted file mode 100644
index 4ff6965..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultDeserializer.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/***********************************************************************************************************************
- * 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.transferenvelope;
-
-import java.io.IOException;
-import java.nio.channels.ReadableByteChannel;
-
-import eu.stratosphere.nephele.io.channels.Buffer;
-import eu.stratosphere.nephele.io.channels.ChannelID;
-import eu.stratosphere.nephele.jobgraph.JobID;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProvider;
-import eu.stratosphere.nephele.taskmanager.bufferprovider.BufferProviderBroker;
-
-public final class DefaultDeserializer extends AbstractDeserializer {
-
- private final BufferProviderBroker bufferProviderBroker;
-
- private BufferProvider bufferProvider = null;
-
- private JobID lastDeserializedJobID = null;
-
- private ChannelID lastDeserializedSourceID = null;
-
- public DefaultDeserializer(final BufferProviderBroker bufferProviderBroker) {
- this.bufferProviderBroker = bufferProviderBroker;
- }
-
-
- @Override
- protected boolean readBufferData(final ReadableByteChannel readableByteChannel) throws IOException,
- NoBufferAvailableException {
-
- if (getBuffer() == null) {
-
- // Find buffer provider for this channel
- if (!getDeserializedJobID().equals(this.lastDeserializedJobID)
- || !getDeserializedSourceID().equals(this.lastDeserializedSourceID)) {
-
- try {
- this.bufferProvider = this.bufferProviderBroker.getBufferProvider(getDeserializedJobID(),
- getDeserializedSourceID());
- } catch (InterruptedException e) {
- return true;
- }
-
- this.lastDeserializedJobID = getDeserializedJobID();
- this.lastDeserializedSourceID = getDeserializedSourceID();
- }
-
- final Buffer buf = this.bufferProvider.requestEmptyBuffer(getSizeOfBuffer());
-
- if (buf == null) {
- throw new NoBufferAvailableException(this.bufferProvider);
- }
-
- setBuffer(buf);
-
- } else {
-
- final Buffer buffer = getBuffer();
-
- final int bytesWritten = buffer.write(readableByteChannel);
-
- if (!buffer.hasRemaining()) {
- // We are done, the buffer has been fully read
- buffer.flip();
- return false;
- } else {
- if (bytesWritten == -1) {
- throw new IOException("Deserialization error: Expected at least " + buffer.remaining()
- + " more bytes to follow");
- }
- }
- }
-
- return true;
- }
-
- public BufferProvider getBufferProvider() {
-
- return this.bufferProvider;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializer.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializer.java
deleted file mode 100644
index 34be592..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/DefaultSerializer.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/***********************************************************************************************************************
- * 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.transferenvelope;
-
-import java.io.IOException;
-import java.nio.channels.WritableByteChannel;
-
-import eu.stratosphere.nephele.io.channels.Buffer;
-
-/**
- * This class is the default implementation to serialize a {@link TransferEnvelope} into a byte stream. In case the
- * transfer envelope contains a buffer, this implementation copies the buffer's data into the byte stream.
- *
- */
-public class DefaultSerializer extends AbstractSerializer {
-
-
- @Override
- protected boolean writeBufferData(final WritableByteChannel writableByteChannel, final Buffer buffer)
- throws IOException {
-
- buffer.writeTo(writableByteChannel);
-
- return buffer.hasRemaining();
- }
-
-}
[02/30] git commit: Fix maven enforcer plugin warnings in Eclipse
Posted by rm...@apache.org.
Fix maven enforcer plugin warnings in Eclipse
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/2a9ed093
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/2a9ed093
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/2a9ed093
Branch: refs/heads/master
Commit: 2a9ed093f9b462956af87ef09c1d5f67b89f9e1c
Parents: 2b0baea
Author: StephanEwen <st...@tu-berlin.de>
Authored: Fri Jun 6 15:18:19 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Fri Jun 6 15:59:22 2014 +0200
----------------------------------------------------------------------
pom.xml | 15 ++++++++++++++-
1 file changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2a9ed093/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f473b56..96acd1d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -541,7 +541,20 @@
</goals>
</pluginExecutionFilter>
<action>
- <ignore></ignore>
+ <ignore />
+ </action>
+ </pluginExecution>
+ <pluginExecution>
+ <pluginExecutionFilter>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-enforcer-plugin</artifactId>
+ <versionRange>[1.0.0,)</versionRange>
+ <goals>
+ <goal>enforce</goal>
+ </goals>
+ </pluginExecutionFilter>
+ <action>
+ <ignore />
</action>
</pluginExecution>
</pluginExecutions>
[28/30] git commit: Merge cleanups
Posted by rm...@apache.org.
Merge cleanups
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/24d00598
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/24d00598
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/24d00598
Branch: refs/heads/master
Commit: 24d00598ab85186dd75f263563273cb3402304c7
Parents: b0f8ba0
Author: StephanEwen <st...@tu-berlin.de>
Authored: Thu Jun 5 15:45:49 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Sat Jun 7 09:41:22 2014 +0200
----------------------------------------------------------------------
stratosphere-core/src/main/java/eu/stratosphere/types/Record.java | 2 +-
.../java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java | 1 -
2 files changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/24d00598/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
----------------------------------------------------------------------
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java b/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
index 5b06547..73671fa 100644
--- a/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
+++ b/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java
@@ -70,7 +70,7 @@ public final class Record implements Value, CopyableValue<Record> {
private Value[] writeFields; // the cache for objects into which the binary representations are read
- public int binaryLen; // the length of the contents in the binary buffer that is valid
+ private int binaryLen; // the length of the contents in the binary buffer that is valid
private int numFields; // the number of fields in the record
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/24d00598/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
index 29d6853..cc542c0 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/RuntimeEnvironment.java
@@ -201,7 +201,6 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
* @param inputSplitProvider 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,
final MemoryManager memoryManager, final IOManager ioManager,
final InputSplitProvider inputSplitProvider,
[04/30] Replace custom Java NIO TCP/IP code with Netty 4 library
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
new file mode 100644
index 0000000..40aefd3
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
@@ -0,0 +1,857 @@
+/***********************************************************************************************************************
+ * 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 + ")";
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java
new file mode 100644
index 0000000..c424a1f
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/NettyConnectionManagerTest.java
@@ -0,0 +1,196 @@
+/***********************************************************************************************************************
+ * 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");
+ }
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoderTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoderTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoderTest.java
new file mode 100644
index 0000000..ba4edc7
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/OutboundEnvelopeEncoderTest.java
@@ -0,0 +1,97 @@
+/***********************************************************************************************************************
+ * 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();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java
index c9323a6..8cda32f 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/KMeansIterativeNepheleITCase.java
@@ -14,6 +14,7 @@
**********************************************************************************************************************/
package eu.stratosphere.test.broadcastvars;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
import org.apache.log4j.Level;
import eu.stratosphere.api.common.operators.util.UserCodeObjectWrapper;
@@ -22,8 +23,6 @@ import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
import eu.stratosphere.api.java.record.io.CsvInputFormat;
import eu.stratosphere.configuration.Configuration;
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.JobGraphDefinitionException;
import eu.stratosphere.nephele.jobgraph.JobInputVertex;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-tests/src/test/java/eu/stratosphere/test/clients/examples/LocalExecutorITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/clients/examples/LocalExecutorITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/clients/examples/LocalExecutorITCase.java
index 802fd0b..984ecc2 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/clients/examples/LocalExecutorITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/clients/examples/LocalExecutorITCase.java
@@ -23,9 +23,8 @@ import eu.stratosphere.client.LocalExecutor;
import eu.stratosphere.test.recordJobs.wordcount.WordCount;
import eu.stratosphere.test.testdata.WordCountData;
-
public class LocalExecutorITCase {
-
+
@Test
public void testLocalExecutorWithWordCount() {
try {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
index 2eaa8a9..4a60836 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/exampleJavaPrograms/WordCountITCase.java
@@ -20,7 +20,7 @@ import eu.stratosphere.test.util.JavaProgramTestBase;
public class WordCountITCase extends JavaProgramTestBase {
-
+
protected String textPath;
protected String resultPath;
@@ -28,7 +28,6 @@ public class WordCountITCase extends JavaProgramTestBase {
setNumTaskManager(2);
}
-
@Override
protected void preSubmit() throws Exception {
textPath = createTempFile("text.txt", WordCountData.TEXT);
@@ -39,7 +38,7 @@ public class WordCountITCase extends JavaProgramTestBase {
protected void postSubmit() throws Exception {
compareResultsByLinesInMemory(WordCountData.COUNTS, resultPath);
}
-
+
@Override
protected void testProgram() throws Exception {
WordCount.main(new String[] { textPath, resultPath });
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/ConnectedComponentsNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
index 0a63af1..40068b7 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
@@ -16,6 +16,8 @@ package eu.stratosphere.test.iterative.nephele;
import java.io.BufferedReader;
import java.util.Collection;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.test.util.RecordAPITestBase;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
@@ -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.FileOutputFormat;
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.JobGraphDefinitionException;
import eu.stratosphere.nephele.jobgraph.JobInputVertex;
@@ -425,11 +425,11 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
JobGraphUtils.connect(head, intermediate, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
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);
- JobGraphUtils.connect(head, output, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
- JobGraphUtils.connect(tail, fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ JobGraphUtils.connect(head, output, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
+ JobGraphUtils.connect(tail, fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
@@ -567,16 +567,16 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
JobGraphUtils.connect(intermediate, ssJoinIntermediate, ChannelType.NETWORK, DistributionPattern.POINTWISE);
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);
- JobGraphUtils.connect(ssJoinIntermediate, wsTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ JobGraphUtils.connect(ssJoinIntermediate, wsTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
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(wsTail, wsFakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ JobGraphUtils.connect(ssTail, ssFakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
+ JobGraphUtils.connect(wsTail, wsFakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
@@ -695,12 +695,12 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
DistributionPattern.POINTWISE);
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);
- 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);
@@ -815,12 +815,12 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
JobGraphUtils.connect(intermediate, ssJoinIntermediate, ChannelType.NETWORK, DistributionPattern.POINTWISE);
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);
- 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);
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/IterationWithChainingNepheleITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/IterationWithChainingNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/IterationWithChainingNepheleITCase.java
index ef83aef..ef7c9d2 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/IterationWithChainingNepheleITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/IterationWithChainingNepheleITCase.java
@@ -15,6 +15,8 @@ package eu.stratosphere.test.iterative.nephele;
import java.util.Collection;
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.runners.Parameterized;
@@ -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.io.FileOutputFormat;
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.JobGraphDefinitionException;
import eu.stratosphere.nephele.jobgraph.JobInputVertex;
@@ -232,16 +232,16 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
// --------------------------------------------------------------------------------------------------------------
// 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);
- 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(tail, fakeTail, ChannelType.INMEMORY, DistributionPattern.POINTWISE);
+ JobGraphUtils.connect(tail, fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
// --------------------------------------------------------------------------------------------------------------
// 3. INSTANCE SHARING
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountUnionReduceITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountUnionReduceITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountUnionReduceITCase.java
index 624fd9b..30ce102 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountUnionReduceITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/WordCountUnionReduceITCase.java
@@ -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.operators.MapOperator;
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.TokenizeLine;
import eu.stratosphere.test.testdata.WordCountData;
@@ -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/124}
- * @see {@link UnionRecordReader}
- * @see {@link MutableUnionRecordReader}
*/
public class WordCountUnionReduceITCase extends RecordAPITestBase {
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/4cd4a134/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java
new file mode 100644
index 0000000..f5beda4
--- /dev/null
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackNepheleITCase.java
@@ -0,0 +1,286 @@
+/***********************************************************************************************************************
+ * 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);
+ }
+ }
+}
[27/30] git commit: Rename BufferAvailabilityRegistration constants
and add initial LocalBufferPoolTest
Posted by rm...@apache.org.
Rename BufferAvailabilityRegistration constants and add initial LocalBufferPoolTest
Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/6a40d41a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/6a40d41a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/6a40d41a
Branch: refs/heads/master
Commit: 6a40d41ac213772b11267203797ed8c83a937392
Parents: 3beca62
Author: uce <u....@fu-berlin.de>
Authored: Tue Jun 3 14:59:16 2014 +0200
Committer: StephanEwen <st...@tu-berlin.de>
Committed: Sat Jun 7 09:41:22 2014 +0200
----------------------------------------------------------------------
.../network/bufferprovider/BufferProvider.java | 16 +-
.../bufferprovider/DiscardBufferPool.java | 2 +-
.../network/bufferprovider/LocalBufferPool.java | 12 +-
.../network/netty/InboundEnvelopeDecoder.java | 6 +-
.../pact/runtime/test/util/MockEnvironment.java | 2 +-
.../bufferprovider/LocalBufferPoolTest.java | 360 +++++++++++++++++++
.../netty/InboundEnvelopeDecoderTest.java | 16 +-
7 files changed, 381 insertions(+), 33 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6a40d41a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
index d82b427..25c8a7e 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/BufferProvider.java
@@ -68,18 +68,8 @@ public interface BufferProvider {
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;
- }
+ SUCCEEDED_REGISTERED(),
+ FAILED_BUFFER_AVAILABLE(),
+ FAILED_BUFFER_POOL_DESTROYED()
}
}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6a40d41a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java
index 5daa509..9b3d434 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/DiscardBufferPool.java
@@ -41,7 +41,7 @@ public final class DiscardBufferPool implements BufferProvider, BufferRecycler {
@Override
public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
- return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
+ return BufferAvailabilityRegistration.FAILED_BUFFER_POOL_DESTROYED;
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6a40d41a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
index 6285726..933df76 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPool.java
@@ -187,17 +187,17 @@ public final class LocalBufferPool implements BufferProvider {
public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
synchronized (this.buffers) {
if (!this.buffers.isEmpty()) {
- return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_AVAILABLE;
+ return BufferAvailabilityRegistration.FAILED_BUFFER_AVAILABLE;
}
if (this.isDestroyed) {
- return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
+ return BufferAvailabilityRegistration.FAILED_BUFFER_POOL_DESTROYED;
}
this.listeners.add(listener);
}
- return BufferAvailabilityRegistration.REGISTERED;
+ return BufferAvailabilityRegistration.SUCCEEDED_REGISTERED;
}
/**
@@ -300,10 +300,7 @@ public final class LocalBufferPool implements BufferProvider {
this.globalBufferPool.returnBuffer(buffer);
this.numRequestedBuffers--;
- return;
- }
-
- if (!this.listeners.isEmpty()) {
+ } else if (!this.listeners.isEmpty()) {
Buffer availableBuffer = new Buffer(buffer, buffer.size(), this.recycler);
try {
this.listeners.poll().bufferAvailable(availableBuffer);
@@ -311,6 +308,7 @@ public final class LocalBufferPool implements BufferProvider {
this.buffers.add(buffer);
this.buffers.notify();
}
+
} else {
this.buffers.add(buffer);
this.buffers.notify();
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6a40d41a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
index 54f4617..1135577 100644
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoder.java
@@ -124,7 +124,7 @@ public class InboundEnvelopeDecoder extends ChannelInboundHandlerAdapter impleme
}
else if (decoderState == DecoderState.NO_BUFFER_AVAILABLE) {
switch (this.currentBufferProvider.registerBufferAvailabilityListener(this)) {
- case REGISTERED:
+ case SUCCEEDED_REGISTERED:
if (ctx.channel().config().isAutoRead()) {
ctx.channel().config().setAutoRead(false);
@@ -137,10 +137,10 @@ public class InboundEnvelopeDecoder extends ChannelInboundHandlerAdapter impleme
this.stagedBuffer.retain();
return false;
- case NOT_REGISTERED_BUFFER_AVAILABLE:
+ case FAILED_BUFFER_AVAILABLE:
continue;
- case NOT_REGISTERED_BUFFER_POOL_DESTROYED:
+ case FAILED_BUFFER_POOL_DESTROYED:
this.bytesToSkip = skipBytes(in, this.currentBufferRequestSize);
this.currentBufferRequestSize = 0;
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6a40d41a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
index b715a4e..2585a74 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/test/util/MockEnvironment.java
@@ -127,7 +127,7 @@ public class MockEnvironment implements Environment, BufferProvider, LocalBuffer
@Override
public BufferAvailabilityRegistration registerBufferAvailabilityListener(BufferAvailabilityListener listener) {
- return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
+ return BufferAvailabilityRegistration.FAILED_BUFFER_POOL_DESTROYED;
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6a40d41a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolTest.java
new file mode 100644
index 0000000..c7d8d41
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/bufferprovider/LocalBufferPoolTest.java
@@ -0,0 +1,360 @@
+/***********************************************************************************************************************
+ * 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.bufferprovider;
+
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.network.bufferprovider.BufferProvider.BufferAvailabilityRegistration;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.util.Timer;
+import java.util.TimerTask;
+
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+public class LocalBufferPoolTest {
+
+ private final static int NUM_BUFFERS = 2048;
+
+ private final static int BUFFER_SIZE = 1024;
+
+ private final static GlobalBufferPool GLOBAL_BUFFER_POOL = new GlobalBufferPool(NUM_BUFFERS, BUFFER_SIZE);
+
+ private final static RecyclingBufferAvailableAnswer RECYCLING_BUFFER_AVAILABLE_ANSWER = new RecyclingBufferAvailableAnswer();
+
+ @BeforeClass
+ public static void setupGlobalBufferPoolOnce() {
+ Assert.assertEquals("GlobalBufferPool does not have required number of buffers.",
+ NUM_BUFFERS, GLOBAL_BUFFER_POOL.numBuffers());
+ Assert.assertEquals("GlobalBufferPool does not have required number of available buffers.",
+ NUM_BUFFERS, GLOBAL_BUFFER_POOL.numAvailableBuffers());
+ }
+
+ @After
+ public void verifyAllBuffersReturnedToGlobalBufferPool() {
+ Assert.assertEquals("Did not return all buffers to GlobalBufferPool after test.",
+ NUM_BUFFERS, GLOBAL_BUFFER_POOL.numAvailableBuffers());
+ }
+
+ @Test
+ public void testSingleConsumerNonBlockingRequestAndRecycle() throws IOException {
+ final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+ Assert.assertEquals(0, bufferPool.numRequestedBuffers());
+
+ // this request-recycle cycle should only take a single buffer out of
+ // the GlobalBufferPool as it is recycled over and over again
+ for (int numRequested = 0; numRequested < NUM_BUFFERS; numRequested++) {
+ Buffer buffer = bufferPool.requestBuffer(BUFFER_SIZE);
+
+ Assert.assertEquals(BUFFER_SIZE, buffer.size());
+
+ Assert.assertEquals("Expected single buffer request in buffer pool.",
+ 1, bufferPool.numRequestedBuffers());
+ Assert.assertEquals("Expected no available buffer in buffer pool.",
+ 0, bufferPool.numAvailableBuffers());
+
+ buffer.recycleBuffer();
+
+ Assert.assertEquals("Expected single available buffer after recycle.",
+ 1, bufferPool.numAvailableBuffers());
+ }
+
+ bufferPool.destroy();
+ }
+
+ @Test
+ public void testSingleConsumerNonBlockingRequestMoreThanAvailable() throws IOException {
+ final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+ Assert.assertEquals(0, bufferPool.numRequestedBuffers());
+
+ // request all buffers from the buffer pool
+ Buffer[] requestedBuffers = new Buffer[NUM_BUFFERS];
+ for (int i = 0; i < NUM_BUFFERS; i++) {
+ requestedBuffers[i] = bufferPool.requestBuffer(BUFFER_SIZE);
+ }
+
+ Assert.assertEquals("Expected no available buffer in buffer pool.",
+ 0, bufferPool.numAvailableBuffers());
+
+ Assert.assertNull("Expected null return value for buffer request with no available buffer.",
+ bufferPool.requestBuffer(BUFFER_SIZE));
+
+ // recycle all buffers and destroy buffer pool
+ for (Buffer buffer : requestedBuffers) {
+ buffer.recycleBuffer();
+ }
+
+ bufferPool.destroy();
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testSingleConsumerNonBlockingRequestTooLarge() throws IOException {
+ final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+ // request too large buffer for the pool
+ bufferPool.requestBuffer(BUFFER_SIZE * 2);
+ }
+
+ @Test
+ public void testSingleConsumerNonBlockingRequestSmall() throws IOException {
+ final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+ // request smaller buffer and verify size
+ Buffer buffer = bufferPool.requestBuffer(BUFFER_SIZE / 2);
+
+ Assert.assertEquals(BUFFER_SIZE / 2, buffer.size());
+
+ buffer.recycleBuffer();
+
+ bufferPool.destroy();
+ }
+
+ @Test
+ public void testSingleConsumerBlockingRequest() throws Exception {
+ final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+ final Buffer[] requestedBuffers = new Buffer[NUM_BUFFERS];
+ for (int i = 0; i < NUM_BUFFERS; i++) {
+ requestedBuffers[i] = bufferPool.requestBuffer(BUFFER_SIZE);
+ }
+
+ final Buffer[] bufferFromBlockingRequest = new Buffer[1];
+
+ // --------------------------------------------------------------------
+ // 1. blocking call: interrupt thread
+ // --------------------------------------------------------------------
+ Assert.assertEquals(NUM_BUFFERS, bufferPool.numRequestedBuffers());
+ Assert.assertEquals(0, bufferPool.numAvailableBuffers());
+
+ Thread blockingBufferRequestThread = new Thread(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ bufferFromBlockingRequest[0] = bufferPool.requestBufferBlocking(BUFFER_SIZE);
+ Assert.fail("Unexpected return from blocking buffer request.");
+ } catch (IOException e) {
+ Assert.fail("Unexpected IOException during test.");
+ } catch (InterruptedException e) {
+ // expected interruption
+ }
+ }
+ });
+
+ // start blocking request thread, sleep, interrupt blocking request thread
+ blockingBufferRequestThread.start();
+
+ Thread.sleep(500);
+
+ blockingBufferRequestThread.interrupt();
+
+ Assert.assertNull(bufferFromBlockingRequest[0]);
+ Assert.assertEquals(NUM_BUFFERS, bufferPool.numRequestedBuffers());
+ Assert.assertEquals(0, bufferPool.numAvailableBuffers());
+
+ // --------------------------------------------------------------------
+ // 2. blocking call: recycle buffer in different thread
+ // --------------------------------------------------------------------
+ // recycle the buffer soon
+ new Timer().schedule(new TimerTask() {
+ @Override
+ public void run() {
+ requestedBuffers[0].recycleBuffer();
+ }
+ }, 500);
+
+ //
+ try {
+ Buffer buffer = bufferPool.requestBufferBlocking(BUFFER_SIZE);
+ Assert.assertNotNull(buffer);
+
+ buffer.recycleBuffer();
+ } catch (InterruptedException e) {
+ Assert.fail("Unexpected InterruptedException during test.");
+ }
+
+ // recycle remaining buffers
+ for (int i = 1; i < requestedBuffers.length; i++) {
+ requestedBuffers[i].recycleBuffer();
+ }
+
+ bufferPool.destroy();
+ }
+
+ @Test
+ public void testSingleConsumerRecycleAfterDestroy() throws IOException {
+ final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+ Buffer[] requestedBuffers = new Buffer[NUM_BUFFERS];
+ for (int i = 0; i < NUM_BUFFERS; i++) {
+ requestedBuffers[i] = bufferPool.requestBuffer(BUFFER_SIZE);
+ }
+
+ bufferPool.destroy();
+
+ // recycle should return buffers to GlobalBufferPool
+ // => verified in verifyAllBuffersReturned()
+ for (Buffer buffer : requestedBuffers) {
+ buffer.recycleBuffer();
+ }
+ }
+
+ @Test
+ public void testSingleConsumerBufferAvailabilityListenerRegistration() throws Exception {
+ final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+ BufferAvailabilityListener listener = mock(BufferAvailabilityListener.class);
+
+ // recycle buffer when listener mock is called back
+ doAnswer(RECYCLING_BUFFER_AVAILABLE_ANSWER).when(listener).bufferAvailable(Matchers.<Buffer>anyObject());
+
+ // request all buffers of the pool
+ Buffer[] requestedBuffers = new Buffer[NUM_BUFFERS];
+ for (int i = 0; i < NUM_BUFFERS; i++) {
+ requestedBuffers[i] = bufferPool.requestBuffer(BUFFER_SIZE);
+ }
+
+ BufferAvailabilityRegistration registration;
+ // --------------------------------------------------------------------
+ // 1. success
+ // --------------------------------------------------------------------
+ registration = bufferPool.registerBufferAvailabilityListener(listener);
+ Assert.assertEquals(BufferAvailabilityRegistration.SUCCEEDED_REGISTERED, registration);
+
+ // verify call to buffer listener after recycle
+ requestedBuffers[0].recycleBuffer();
+ verify(listener, times(1)).bufferAvailable(Matchers.<Buffer>anyObject());
+
+ Assert.assertEquals("Expected single available buffer after recycle call in mock listener.",
+ 1, bufferPool.numAvailableBuffers());
+
+ // --------------------------------------------------------------------
+ // 2. failure: buffer is available
+ // --------------------------------------------------------------------
+ registration = bufferPool.registerBufferAvailabilityListener(listener);
+ Assert.assertEquals(BufferAvailabilityRegistration.FAILED_BUFFER_AVAILABLE, registration);
+
+ Buffer buffer = bufferPool.requestBuffer(BUFFER_SIZE);
+ Assert.assertNotNull(buffer);
+
+ buffer.recycleBuffer();
+
+ // --------------------------------------------------------------------
+ // 3. failure: buffer pool destroyed
+ // --------------------------------------------------------------------
+ bufferPool.destroy();
+
+ registration = bufferPool.registerBufferAvailabilityListener(listener);
+ Assert.assertEquals(BufferAvailabilityRegistration.FAILED_BUFFER_POOL_DESTROYED, registration);
+
+ // recycle remaining buffers
+ for (int i = 1; i < requestedBuffers.length; i++) {
+ requestedBuffers[i].recycleBuffer();
+ }
+ }
+
+ @Test
+ public void testSingleConsumerReturnExcessBuffers() throws Exception {
+ final LocalBufferPool bufferPool = new LocalBufferPool(GLOBAL_BUFFER_POOL, NUM_BUFFERS);
+
+ // request all buffers of the pool
+ Buffer[] requestedBuffers = new Buffer[NUM_BUFFERS];
+ for (int i = 0; i < NUM_BUFFERS; i++) {
+ requestedBuffers[i] = bufferPool.requestBuffer(BUFFER_SIZE);
+ }
+
+ Assert.assertEquals(NUM_BUFFERS, bufferPool.numRequestedBuffers());
+ Assert.assertEquals(0, bufferPool.numAvailableBuffers());
+
+ // recycle first half of the buffers
+ // => leave requested number of buffers unchanged
+ // => increase available number of buffers
+ for (int i = 0; i < NUM_BUFFERS / 2; i++) {
+ requestedBuffers[i].recycleBuffer();
+ }
+
+ Assert.assertEquals(NUM_BUFFERS, bufferPool.numRequestedBuffers());
+ Assert.assertEquals(NUM_BUFFERS / 2, bufferPool.numAvailableBuffers());
+
+ // reduce designated number of buffers
+ // => available buffers (1/2th) should be returned immediately
+ // => non-available buffers (1/4th) should be returned later
+ bufferPool.setNumDesignatedBuffers((NUM_BUFFERS / 2) - (NUM_BUFFERS / 4));
+
+ Assert.assertEquals(NUM_BUFFERS / 2, bufferPool.numRequestedBuffers());
+ Assert.assertEquals(0, bufferPool.numAvailableBuffers());
+
+ // recycle second half of the buffers
+ // => previously non-available buffers (1/4th) should be returned immediately
+ // => remaining buffers are the available ones (1/4th)
+ for (int i = NUM_BUFFERS / 2; i < NUM_BUFFERS; i++) {
+ requestedBuffers[i].recycleBuffer();
+ }
+
+ Assert.assertEquals("Expected current number of requested buffers to be equal to the number of designated buffers.",
+ bufferPool.numDesignatedBuffers(), bufferPool.numRequestedBuffers());
+
+ Assert.assertEquals("Expected current number of requested and available buffers to be equal, " +
+ "because all requested buffers have been recycled and become available again.",
+ bufferPool.numRequestedBuffers(), bufferPool.numAvailableBuffers());
+
+ // re-request remaining buffers and register buffer availability listener
+ int remaining = bufferPool.numRequestedBuffers();
+ for (int i = 0; i < remaining; i++) {
+ requestedBuffers[i] = bufferPool.requestBuffer(BUFFER_SIZE);
+ }
+
+ BufferAvailabilityListener listener = mock(BufferAvailabilityListener.class);
+ doAnswer(RECYCLING_BUFFER_AVAILABLE_ANSWER).when(listener).bufferAvailable(Matchers.<Buffer>anyObject());
+
+ BufferAvailabilityRegistration registration = bufferPool.registerBufferAvailabilityListener(listener);
+ Assert.assertEquals(BufferAvailabilityRegistration.SUCCEEDED_REGISTERED, registration);
+
+ // reduce number of designated buffers and recycle all buffers
+ bufferPool.setNumDesignatedBuffers(bufferPool.numDesignatedBuffers() - 1);
+
+ for (int i = 0; i < remaining; i++) {
+ requestedBuffers[i].recycleBuffer();
+ }
+
+ Assert.assertEquals(remaining - 1, bufferPool.numRequestedBuffers());
+ Assert.assertEquals(remaining - 1, bufferPool.numAvailableBuffers());
+
+ bufferPool.destroy();
+ }
+
+ // --------------------------------------------------------------------
+
+ private static class RecyclingBufferAvailableAnswer implements Answer<Void> {
+
+ @Override
+ public Void answer(InvocationOnMock invocation) throws Throwable {
+ Buffer buffer = (Buffer) invocation.getArguments()[0];
+ buffer.recycleBuffer();
+
+ return null;
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/6a40d41a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
index 40aefd3..f695979 100644
--- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java
@@ -83,7 +83,7 @@ public class InboundEnvelopeDecoderTest {
ByteBuf buf = encode(ch, envelopes);
when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
- .thenReturn(BufferAvailabilityRegistration.REGISTERED);
+ .thenReturn(BufferAvailabilityRegistration.SUCCEEDED_REGISTERED);
Buffer buffer = allocBuffer(envelopes[2].getBuffer().size());
@@ -161,7 +161,7 @@ public class InboundEnvelopeDecoderTest {
.thenReturn(null);
when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
- .thenReturn(BufferAvailabilityRegistration.REGISTERED);
+ .thenReturn(BufferAvailabilityRegistration.SUCCEEDED_REGISTERED);
// --------------------------------------------------------------------
@@ -196,7 +196,7 @@ public class InboundEnvelopeDecoderTest {
Envelope[] envelopes = new Envelope[]{nextEnvelope(true), nextEnvelope()};
when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
- .thenReturn(BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_AVAILABLE);
+ .thenReturn(BufferAvailabilityRegistration.FAILED_BUFFER_AVAILABLE);
when(this.bufferProvider.requestBuffer(anyInt()))
.thenReturn(null)
@@ -223,7 +223,7 @@ public class InboundEnvelopeDecoderTest {
.thenReturn(null);
when(this.bufferProvider.registerBufferAvailabilityListener(Matchers.<BufferAvailabilityListener>anyObject()))
- .thenReturn(BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED);
+ .thenReturn(BufferAvailabilityRegistration.FAILED_BUFFER_POOL_DESTROYED);
// --------------------------------------------------------------------
@@ -738,15 +738,15 @@ public class InboundEnvelopeDecoderTest {
public BufferAvailabilityRegistration answer(InvocationOnMock invocation) throws Throwable {
if (this.random.nextBoolean()) {
this.isRegistered = true;
- return BufferAvailabilityRegistration.REGISTERED;
+ return BufferAvailabilityRegistration.SUCCEEDED_REGISTERED;
}
else if (this.random.nextBoolean()) {
this.bufferRequestAnswer.forceBufferAvailable();
- return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_AVAILABLE;
+ return BufferAvailabilityRegistration.FAILED_BUFFER_AVAILABLE;
}
else {
this.numSkipped++;
- return BufferAvailabilityRegistration.NOT_REGISTERED_BUFFER_POOL_DESTROYED;
+ return BufferAvailabilityRegistration.FAILED_BUFFER_POOL_DESTROYED;
}
}
@@ -757,7 +757,7 @@ public class InboundEnvelopeDecoderTest {
for (Envelope env : envelopes) {
if (env.getBuffer() != null) {
- // skip envelope if returned NOT_REGISTERED_BUFFER_POOL_DESTROYED
+ // skip envelope if returned FAILED_BUFFER_POOL_DESTROYED
if (!this.random.nextBoolean() && !this.random.nextBoolean() && !this.random.nextBoolean()) {
continue;
}
[09/30] Offer buffer-oriented API for I/O (#25)
Posted by rm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/DefaultChannelSelectorTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/DefaultChannelSelectorTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/DefaultChannelSelectorTest.java
new file mode 100644
index 0000000..890b34f
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/DefaultChannelSelectorTest.java
@@ -0,0 +1,47 @@
+/***********************************************************************************************************************
+ * 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;
+
+import static org.junit.Assert.assertEquals;
+
+import eu.stratosphere.runtime.io.api.RoundRobinChannelSelector;
+import org.junit.Test;
+
+import eu.stratosphere.core.io.StringRecord;
+
+/**
+ * This class checks the functionality of the {@link RoundRobinChannelSelector} class.
+ *
+ */
+public class DefaultChannelSelectorTest {
+
+ /**
+ * This test checks the channel selection
+ */
+ @Test
+ public void channelSelect() {
+
+ final StringRecord dummyRecord = new StringRecord("abc");
+ final RoundRobinChannelSelector<StringRecord> selector = new RoundRobinChannelSelector<StringRecord>();
+ // Test with two channels
+ final int numberOfOutputChannels = 2;
+ int[] selectedChannels = selector.selectChannels(dummyRecord, numberOfOutputChannels);
+ assertEquals(1, selectedChannels.length);
+ assertEquals(1, selectedChannels[0]);
+ selectedChannels = selector.selectChannels(dummyRecord, numberOfOutputChannels);
+ assertEquals(1, selectedChannels.length);
+ assertEquals(0, selectedChannels[0]);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/library/FileLineReadWriteTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/library/FileLineReadWriteTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/library/FileLineReadWriteTest.java
new file mode 100644
index 0000000..17c2f58
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/library/FileLineReadWriteTest.java
@@ -0,0 +1,136 @@
+/***********************************************************************************************************************
+ * 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.library;
+
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.powermock.api.mockito.PowerMockito.whenNew;
+
+import java.io.File;
+import java.io.IOException;
+
+import eu.stratosphere.runtime.io.api.RecordWriter;
+import eu.stratosphere.nephele.util.FileLineReader;
+import eu.stratosphere.nephele.util.FileLineWriter;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.MockitoAnnotations;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.powermock.reflect.Whitebox;
+
+import eu.stratosphere.configuration.Configuration;
+import eu.stratosphere.core.fs.FileInputSplit;
+import eu.stratosphere.core.fs.Path;
+import eu.stratosphere.core.io.StringRecord;
+import eu.stratosphere.nephele.execution.Environment;
+import eu.stratosphere.runtime.io.api.RecordReader;
+import eu.stratosphere.nephele.template.InputSplitProvider;
+
+/**
+ * This class checks the functionality of the {@link eu.stratosphere.nephele.util.FileLineReader} and the {@link eu.stratosphere.nephele.util.FileLineWriter} class.
+ *
+ */
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(FileLineReader.class)
+public class FileLineReadWriteTest {
+
+ @Mock
+ private Environment environment;
+
+ @Mock
+ private Configuration conf;
+
+ @Mock
+ private RecordReader<StringRecord> recordReader;
+
+ @Mock
+ private RecordWriter<StringRecord> recordWriter;
+
+ @Mock
+ private InputSplitProvider inputSplitProvider;
+
+ private File file = new File("./tmp");
+
+ /**
+ * Set up mocks
+ *
+ * @throws IOException
+ */
+ @Before
+ public void before() throws Exception {
+
+ MockitoAnnotations.initMocks(this);
+ }
+
+ /**
+ * remove the temporary file
+ */
+ @After
+ public void after() {
+ this.file.delete();
+ }
+
+ /**
+ * Tests the read and write methods
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testReadWrite() throws Exception {
+
+ this.file.createNewFile();
+ FileLineWriter writer = new FileLineWriter();
+ Whitebox.setInternalState(writer, "environment", this.environment);
+ Whitebox.setInternalState(writer, "input", this.recordReader);
+ when(this.environment.getTaskConfiguration()).thenReturn(this.conf);
+
+ when(this.conf.getString("outputPath", null)).thenReturn(this.file.toURI().toString());
+ when(this.recordReader.hasNext()).thenReturn(true, true, true, false);
+ StringRecord in = new StringRecord("abc");
+ try {
+ when(this.recordReader.next()).thenReturn(in);
+ } catch (IOException e) {
+ fail();
+ e.printStackTrace();
+ } catch (InterruptedException e) {
+ fail();
+ e.printStackTrace();
+ }
+ writer.invoke();
+
+ final FileInputSplit split = new FileInputSplit(0, new Path(this.file.toURI().toString()), 0,
+ this.file.length(), null);
+ when(this.environment.getInputSplitProvider()).thenReturn(this.inputSplitProvider);
+ when(this.inputSplitProvider.getNextInputSplit()).thenReturn(split, (FileInputSplit) null);
+
+ FileLineReader reader = new FileLineReader();
+ Whitebox.setInternalState(reader, "environment", this.environment);
+ Whitebox.setInternalState(reader, "output", this.recordWriter);
+ StringRecord record = mock(StringRecord.class);
+
+ whenNew(StringRecord.class).withNoArguments().thenReturn(record);
+
+ reader.invoke();
+
+ // verify the correct bytes have been written and read
+ verify(record, times(3)).set(in.getBytes());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java
new file mode 100644
index 0000000..b8914a8
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/envelope/EnvelopeReaderWriterTest.java
@@ -0,0 +1,394 @@
+/***********************************************************************************************************************
+ * 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;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/DataInputOutputSerializerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/DataInputOutputSerializerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/DataInputOutputSerializerTest.java
new file mode 100644
index 0000000..55c0243
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/DataInputOutputSerializerTest.java
@@ -0,0 +1,115 @@
+/***********************************************************************************************************************
+ * 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.serialization;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestType;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestTypeFactory;
+import eu.stratosphere.runtime.io.serialization.types.Util;
+import junit.framework.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+
+public class DataInputOutputSerializerTest {
+
+ @Test
+ public void testWrapAsByteBuffer() {
+ SerializationTestType randomInt = Util.randomRecord(SerializationTestTypeFactory.INT);
+
+ DataOutputSerializer serializer = new DataOutputSerializer(randomInt.length());
+ MemorySegment segment = new MemorySegment(new byte[randomInt.length()]);
+
+ try {
+ // empty buffer, read buffer should be empty
+ ByteBuffer wrapper = serializer.wrapAsByteBuffer();
+
+ Assert.assertEquals(0, wrapper.position());
+ Assert.assertEquals(0, wrapper.limit());
+
+ // write to data output, read buffer should still be empty
+ randomInt.write(serializer);
+
+ Assert.assertEquals(0, wrapper.position());
+ Assert.assertEquals(0, wrapper.limit());
+
+ // get updated read buffer, read buffer should contain written data
+ wrapper = serializer.wrapAsByteBuffer();
+
+ Assert.assertEquals(0, wrapper.position());
+ Assert.assertEquals(randomInt.length(), wrapper.limit());
+
+ // clear data output, read buffer should still contain written data
+ serializer.clear();
+
+ Assert.assertEquals(0, wrapper.position());
+ Assert.assertEquals(randomInt.length(), wrapper.limit());
+
+ // get updated read buffer, should be empty
+ wrapper = serializer.wrapAsByteBuffer();
+
+ Assert.assertEquals(0, wrapper.position());
+ Assert.assertEquals(0, wrapper.limit());
+
+ // write to data output and read back to memory
+ randomInt.write(serializer);
+ wrapper = serializer.wrapAsByteBuffer();
+
+ segment.put(0, wrapper, randomInt.length());
+
+ Assert.assertEquals(randomInt.length(), wrapper.position());
+ Assert.assertEquals(randomInt.length(), wrapper.limit());
+ } catch (IOException e) {
+ e.printStackTrace();
+ Assert.fail("Test encountered an unexpected exception.");
+ }
+ }
+
+ @Test
+ public void testRandomValuesWriteRead() {
+ final int numElements = 100000;
+ final ArrayDeque<SerializationTestType> reference = new ArrayDeque<SerializationTestType>();
+
+ DataOutputSerializer serializer = new DataOutputSerializer(1);
+
+ for (SerializationTestType value : Util.randomRecords(numElements)) {
+ reference.add(value);
+
+ try {
+ value.write(serializer);
+ } catch (IOException e) {
+ e.printStackTrace();
+ Assert.fail("Test encountered an unexpected exception.");
+ }
+ }
+
+ DataInputDeserializer deserializer = new DataInputDeserializer(serializer.wrapAsByteBuffer());
+
+ for (SerializationTestType expected : reference) {
+ try {
+ SerializationTestType actual = expected.getClass().newInstance();
+ actual.read(deserializer);
+
+ Assert.assertEquals(expected, actual);
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail("Test encountered an unexpected exception.");
+ }
+ }
+
+ reference.clear();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/PagedViewsTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/PagedViewsTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/PagedViewsTest.java
new file mode 100644
index 0000000..817c0e6
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/PagedViewsTest.java
@@ -0,0 +1,160 @@
+/***********************************************************************************************************************
+ * 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.serialization;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.nephele.services.memorymanager.AbstractPagedInputView;
+import eu.stratosphere.nephele.services.memorymanager.AbstractPagedOutputView;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestType;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestTypeFactory;
+import eu.stratosphere.runtime.io.serialization.types.Util;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.*;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class PagedViewsTest {
+
+ @Test
+ public void testSequenceOfIntegersWithAlignedBuffers() {
+ try {
+ final int NUM_INTS = 1000000;
+
+ testSequenceOfTypes(Util.randomRecords(NUM_INTS, SerializationTestTypeFactory.INT), 2048);
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail("Test encountered an unexpected exception.");
+ }
+ }
+
+ @Test
+ public void testSequenceOfIntegersWithUnalignedBuffers() {
+ try {
+ final int NUM_INTS = 1000000;
+
+ testSequenceOfTypes(Util.randomRecords(NUM_INTS, SerializationTestTypeFactory.INT), 2047);
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail("Test encountered an unexpected exception.");
+ }
+ }
+
+ @Test
+ public void testRandomTypes() {
+ try {
+ final int NUM_TYPES = 100000;
+
+ // test with an odd buffer size to force many unaligned cases
+ testSequenceOfTypes(Util.randomRecords(NUM_TYPES), 57);
+
+ } catch (Exception e) {
+ e.printStackTrace();
+ fail("Test encountered an unexpected exception.");
+ }
+ }
+
+ private static void testSequenceOfTypes(Iterable<SerializationTestType> sequence, int segmentSize) throws Exception {
+
+ List<SerializationTestType> elements = new ArrayList<SerializationTestType>(512);
+ TestOutputView outView = new TestOutputView(segmentSize);
+
+ // write
+ for (SerializationTestType type : sequence) {
+ // serialize the record
+ type.write(outView);
+ elements.add(type);
+ }
+ outView.close();
+
+ // check the records
+ TestInputView inView = new TestInputView(outView.segments);
+
+ for (SerializationTestType reference : elements) {
+ SerializationTestType result = reference.getClass().newInstance();
+ result.read(inView);
+ assertEquals(reference, result);
+ }
+ }
+
+ // ============================================================================================
+
+ private static final class SegmentWithPosition {
+
+ private final MemorySegment segment;
+ private final int position;
+
+ public SegmentWithPosition(MemorySegment segment, int position) {
+ this.segment = segment;
+ this.position = position;
+ }
+ }
+
+ private static final class TestOutputView extends AbstractPagedOutputView {
+
+ private final List<SegmentWithPosition> segments = new ArrayList<SegmentWithPosition>();
+
+ private final int segmentSize;
+
+ private TestOutputView(int segmentSize) {
+ super(new MemorySegment(new byte[segmentSize]), segmentSize, 0);
+
+ this.segmentSize = segmentSize;
+ }
+
+ @Override
+ protected MemorySegment nextSegment(MemorySegment current, int positionInCurrent) throws IOException {
+ segments.add(new SegmentWithPosition(current, positionInCurrent));
+ return new MemorySegment(new byte[segmentSize]);
+ }
+
+ public void close() {
+ segments.add(new SegmentWithPosition(getCurrentSegment(), getCurrentPositionInSegment()));
+ }
+ }
+
+ private static final class TestInputView extends AbstractPagedInputView {
+
+ private final List<SegmentWithPosition> segments;
+
+ private int num;
+
+
+ private TestInputView(List<SegmentWithPosition> segments) {
+ super(segments.get(0).segment, segments.get(0).position, 0);
+
+ this.segments = segments;
+ this.num = 0;
+ }
+
+ @Override
+ protected MemorySegment nextSegment(MemorySegment current) throws IOException {
+ num++;
+ if (num < segments.size()) {
+ return segments.get(num).segment;
+ } else {
+ return null;
+ }
+ }
+
+ @Override
+ protected int getLimitForSegment(MemorySegment segment) {
+ return segments.get(num).position;
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializationTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializationTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializationTest.java
new file mode 100644
index 0000000..094e597
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializationTest.java
@@ -0,0 +1,164 @@
+/***********************************************************************************************************************
+ * 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.serialization;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.serialization.RecordDeserializer.DeserializationResult;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestType;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestTypeFactory;
+import eu.stratosphere.runtime.io.serialization.types.Util;
+import junit.framework.Assert;
+import org.junit.Test;
+
+import java.util.ArrayDeque;
+
+public class SpanningRecordSerializationTest {
+
+ @Test
+ public void testIntRecordsSpanningMultipleSegments() {
+ final int SEGMENT_SIZE = 1;
+ final int NUM_VALUES = 10;
+
+ try {
+ test(Util.randomRecords(NUM_VALUES, SerializationTestTypeFactory.INT), SEGMENT_SIZE);
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail("Test encountered an unexpected exception.");
+ }
+ }
+
+ @Test
+ public void testIntRecordsWithAlignedBuffers () {
+ final int SEGMENT_SIZE = 64;
+ final int NUM_VALUES = 64;
+
+ try {
+ test(Util.randomRecords(NUM_VALUES, SerializationTestTypeFactory.INT), SEGMENT_SIZE);
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail("Test encountered an unexpected exception.");
+ }
+ }
+
+ @Test
+ public void testIntRecordsWithUnalignedBuffers () {
+ final int SEGMENT_SIZE = 31;
+ final int NUM_VALUES = 248;
+
+ try {
+ test(Util.randomRecords(NUM_VALUES, SerializationTestTypeFactory.INT), SEGMENT_SIZE);
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail("Test encountered an unexpected exception.");
+ }
+ }
+
+ @Test
+ public void testRandomRecords () {
+ final int SEGMENT_SIZE = 127;
+ final int NUM_VALUES = 10000;
+
+ try {
+ test(Util.randomRecords(NUM_VALUES), SEGMENT_SIZE);
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail("Test encountered an unexpected exception.");
+ }
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ /**
+ * Iterates over the provided records and tests whether {@link SpanningRecordSerializer} and {@link AdaptiveSpanningRecordDeserializer}
+ * interact as expected.
+ * <p>
+ * Only a single {@link MemorySegment} will be allocated.
+ *
+ * @param records records to test
+ * @param segmentSize size for the {@link MemorySegment}
+ */
+ private void test (Util.MockRecords records, int segmentSize) throws Exception {
+ final int SERIALIZATION_OVERHEAD = 4; // length encoding
+
+ final RecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
+ final RecordDeserializer<SerializationTestType> deserializer = new AdaptiveSpanningRecordDeserializer<SerializationTestType>();
+
+ final Buffer buffer = new Buffer(new MemorySegment(new byte[segmentSize]), segmentSize, null);
+
+ final ArrayDeque<SerializationTestType> serializedRecords = new ArrayDeque<SerializationTestType>();
+
+ // -------------------------------------------------------------------------------------------------------------
+
+ serializer.setNextBuffer(buffer);
+
+ int numBytes = 0;
+ int numRecords = 0;
+ for (SerializationTestType record : records) {
+
+ serializedRecords.add(record);
+
+ numRecords++;
+ numBytes += record.length() + SERIALIZATION_OVERHEAD;
+
+ // serialize record
+ if (serializer.addRecord(record).isFullBuffer()) {
+ // buffer is full => start deserializing
+ deserializer.setNextMemorySegment(serializer.getCurrentBuffer().getMemorySegment(), segmentSize);
+
+ while (!serializedRecords.isEmpty()) {
+ SerializationTestType expected = serializedRecords.poll();
+ SerializationTestType actual = expected.getClass().newInstance();
+
+ if (deserializer.getNextRecord(actual).isFullRecord()) {
+ Assert.assertEquals(expected, actual);
+ numRecords--;
+ } else {
+ serializedRecords.addFirst(expected);
+ break;
+ }
+ }
+
+ while (serializer.setNextBuffer(buffer).isFullBuffer()) {
+ deserializer.setNextMemorySegment(serializer.getCurrentBuffer().getMemorySegment(), segmentSize);
+ }
+
+
+
+ }
+ }
+
+ // deserialize left over records
+ deserializer.setNextMemorySegment(serializer.getCurrentBuffer().getMemorySegment(), (numBytes % segmentSize));
+
+ serializer.clear();
+
+ while (!serializedRecords.isEmpty()) {
+ SerializationTestType expected = serializedRecords.poll();
+
+ SerializationTestType actual = expected.getClass().newInstance();
+ DeserializationResult result = deserializer.getNextRecord(actual);
+
+ Assert.assertTrue(result.isFullRecord());
+ Assert.assertEquals(expected, actual);
+ numRecords--;
+ }
+
+
+ // assert that all records have been serialized and deserialized
+ Assert.assertEquals(0, numRecords);
+ Assert.assertFalse(serializer.hasData());
+ Assert.assertFalse(deserializer.hasUnfinishedData());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializerTest.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializerTest.java
new file mode 100644
index 0000000..637b7d5
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializerTest.java
@@ -0,0 +1,219 @@
+/***********************************************************************************************************************
+ * 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.serialization;
+
+import eu.stratosphere.core.memory.MemorySegment;
+import eu.stratosphere.runtime.io.Buffer;
+import eu.stratosphere.runtime.io.serialization.RecordSerializer.SerializationResult;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestType;
+import eu.stratosphere.runtime.io.serialization.types.SerializationTestTypeFactory;
+import eu.stratosphere.runtime.io.serialization.types.Util;
+import junit.framework.Assert;
+import org.junit.Test;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class SpanningRecordSerializerTest {
+
+ @Test
+ public void testHasData() {
+ final int SEGMENT_SIZE = 16;
+
+ final SpanningRecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
+ final Buffer buffer = new Buffer(new MemorySegment(new byte[SEGMENT_SIZE]), SEGMENT_SIZE, null);
+ final SerializationTestType randomIntRecord = Util.randomRecord(SerializationTestTypeFactory.INT);
+
+ Assert.assertFalse(serializer.hasData());
+
+ try {
+ serializer.addRecord(randomIntRecord);
+ Assert.assertTrue(serializer.hasData());
+
+ serializer.setNextBuffer(buffer);
+ Assert.assertTrue(serializer.hasData());
+
+ serializer.clear();
+ Assert.assertFalse(serializer.hasData());
+
+ serializer.setNextBuffer(buffer);
+
+ serializer.addRecord(randomIntRecord);
+ Assert.assertTrue(serializer.hasData());
+
+ serializer.addRecord(randomIntRecord);
+ Assert.assertTrue(serializer.hasData());
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+
+ }
+
+ @Test
+ public void testEmptyRecords() {
+ final int SEGMENT_SIZE = 11;
+
+ final SpanningRecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
+ final Buffer buffer = new Buffer(new MemorySegment(new byte[SEGMENT_SIZE]), SEGMENT_SIZE, null);
+
+ try {
+ Assert.assertEquals(SerializationResult.FULL_RECORD, serializer.setNextBuffer(buffer));
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+
+ try {
+ SerializationTestType emptyRecord = new SerializationTestType() {
+ @Override
+ public SerializationTestType getRandom(Random rnd) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public int length() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ }
+
+ @Override
+ public int hashCode() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ throw new UnsupportedOperationException();
+ }
+ };
+
+ SerializationResult result = serializer.addRecord(emptyRecord);
+ Assert.assertEquals(SerializationResult.FULL_RECORD, result);
+
+ result = serializer.addRecord(emptyRecord);
+ Assert.assertEquals(SerializationResult.FULL_RECORD, result);
+
+ result = serializer.addRecord(emptyRecord);
+ Assert.assertEquals(SerializationResult.PARTIAL_RECORD_MEMORY_SEGMENT_FULL, result);
+
+ result = serializer.setNextBuffer(buffer);
+ Assert.assertEquals(SerializationResult.FULL_RECORD, result);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+
+ @Test
+ public void testIntRecordsSpanningMultipleSegments() {
+ final int SEGMENT_SIZE = 1;
+ final int NUM_VALUES = 10;
+
+ try {
+ test(Util.randomRecords(NUM_VALUES, SerializationTestTypeFactory.INT), SEGMENT_SIZE);
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail("Test encountered an unexpected exception.");
+ }
+ }
+
+ @Test
+ public void testIntRecordsWithAlignedSegments() {
+ final int SEGMENT_SIZE = 64;
+ final int NUM_VALUES = 64;
+
+ try {
+ test(Util.randomRecords(NUM_VALUES, SerializationTestTypeFactory.INT), SEGMENT_SIZE);
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail("Test encountered an unexpected exception.");
+ }
+ }
+
+ @Test
+ public void testIntRecordsWithUnalignedSegments() {
+ final int SEGMENT_SIZE = 31;
+ final int NUM_VALUES = 248; // least common multiple => last record should align
+
+ try {
+ test(Util.randomRecords(NUM_VALUES, SerializationTestTypeFactory.INT), SEGMENT_SIZE);
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail("Test encountered an unexpected exception.");
+ }
+ }
+
+ @Test
+ public void testRandomRecords() {
+ final int SEGMENT_SIZE = 127;
+ final int NUM_VALUES = 100000;
+
+ try {
+ test(Util.randomRecords(NUM_VALUES), SEGMENT_SIZE);
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail("Test encountered an unexpected exception.");
+ }
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+
+ /**
+ * Iterates over the provided records and tests whether the {@link SpanningRecordSerializer} returns the expected
+ * {@link SerializationResult} values.
+ * <p>
+ * Only a single {@link MemorySegment} will be allocated.
+ *
+ * @param records records to test
+ * @param segmentSize size for the {@link MemorySegment}
+ */
+ private void test(Util.MockRecords records, int segmentSize) throws Exception {
+ final int SERIALIZATION_OVERHEAD = 4; // length encoding
+
+ final SpanningRecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
+ final Buffer buffer = new Buffer(new MemorySegment(new byte[segmentSize]), segmentSize, null);
+
+ // -------------------------------------------------------------------------------------------------------------
+
+ serializer.setNextBuffer(buffer);
+
+ int numBytes = 0;
+ for (SerializationTestType record : records) {
+ SerializationResult result = serializer.addRecord(record);
+ numBytes += record.length() + SERIALIZATION_OVERHEAD;
+
+ if (numBytes < segmentSize) {
+ Assert.assertEquals(SerializationResult.FULL_RECORD, result);
+ } else if (numBytes == segmentSize) {
+ Assert.assertEquals(SerializationResult.FULL_RECORD_MEMORY_SEGMENT_FULL, result);
+ serializer.setNextBuffer(buffer);
+ numBytes = 0;
+ } else {
+ Assert.assertEquals(SerializationResult.PARTIAL_RECORD_MEMORY_SEGMENT_FULL, result);
+
+ while (result.isFullBuffer()) {
+ numBytes -= segmentSize;
+ result = serializer.setNextBuffer(buffer);
+ }
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/AsciiStringType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/AsciiStringType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/AsciiStringType.java
new file mode 100644
index 0000000..7aadc7c
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/AsciiStringType.java
@@ -0,0 +1,77 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class AsciiStringType implements SerializationTestType {
+
+ private static final int MAX_LEN = 1500;
+
+ public String value;
+
+ public AsciiStringType() {
+ this.value = "";
+ }
+
+ private AsciiStringType(String value) {
+ this.value = value;
+ }
+
+ @Override
+ public AsciiStringType getRandom(Random rnd) {
+ final StringBuilder bld = new StringBuilder();
+ final int len = rnd.nextInt(MAX_LEN + 1);
+
+ for (int i = 0; i < len; i++) {
+ // 1--127
+ bld.append((char) (rnd.nextInt(126) + 1));
+ }
+
+ return new AsciiStringType(bld.toString());
+ }
+
+ @Override
+ public int length() {
+ return value.getBytes().length + 2;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeUTF(this.value);
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ this.value = in.readUTF();
+ }
+
+ @Override
+ public int hashCode() {
+ return this.value.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof AsciiStringType) {
+ AsciiStringType other = (AsciiStringType) obj;
+ return this.value.equals(other.value);
+ } else {
+ return false;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/BooleanType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/BooleanType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/BooleanType.java
new file mode 100644
index 0000000..32b2ba3
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/BooleanType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class BooleanType implements SerializationTestType {
+
+ private boolean value;
+
+ public BooleanType() {
+ this.value = false;
+ }
+
+ private BooleanType(boolean value) {
+ this.value = value;
+ }
+
+ @Override
+ public BooleanType getRandom(Random rnd) {
+ return new BooleanType(rnd.nextBoolean());
+ }
+
+ @Override
+ public int length() {
+ return 1;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeBoolean(this.value);
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ this.value = in.readBoolean();
+ }
+
+ @Override
+ public int hashCode() {
+ return this.value ? 1 : 0;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof BooleanType) {
+ BooleanType other = (BooleanType) obj;
+ return this.value == other.value;
+ } else {
+ return false;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteArrayType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteArrayType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteArrayType.java
new file mode 100644
index 0000000..cb29a5c
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteArrayType.java
@@ -0,0 +1,76 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Random;
+
+public class ByteArrayType implements SerializationTestType {
+
+ private static final int MAX_LEN = 512 * 15;
+
+ private byte[] data;
+
+ public ByteArrayType() {
+ this.data = new byte[0];
+ }
+
+ public ByteArrayType(byte[] data) {
+ this.data = data;
+ }
+
+ @Override
+ public ByteArrayType getRandom(Random rnd) {
+ final int len = rnd.nextInt(MAX_LEN) + 1;
+ final byte[] data = new byte[len];
+ rnd.nextBytes(data);
+ return new ByteArrayType(data);
+ }
+
+ @Override
+ public int length() {
+ return data.length + 4;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeInt(this.data.length);
+ out.write(this.data);
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ final int len = in.readInt();
+ this.data = new byte[len];
+ in.readFully(this.data);
+ }
+
+ @Override
+ public int hashCode() {
+ return Arrays.hashCode(this.data);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof ByteArrayType) {
+ ByteArrayType other = (ByteArrayType) obj;
+ return Arrays.equals(this.data, other.data);
+ } else {
+ return false;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteSubArrayType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteSubArrayType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteSubArrayType.java
new file mode 100644
index 0000000..2b683d2
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteSubArrayType.java
@@ -0,0 +1,91 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Random;
+
+public class ByteSubArrayType implements SerializationTestType {
+
+ private static final int MAX_LEN = 512;
+
+ private final byte[] data;
+
+ private int len;
+
+ public ByteSubArrayType() {
+ this.data = new byte[MAX_LEN];
+ this.len = 0;
+ }
+
+ @Override
+ public ByteSubArrayType getRandom(Random rnd) {
+ final int len = rnd.nextInt(MAX_LEN) + 1;
+ final ByteSubArrayType t = new ByteSubArrayType();
+ t.len = len;
+
+ final byte[] data = t.data;
+ for (int i = 0; i < len; i++) {
+ data[i] = (byte) rnd.nextInt(256);
+ }
+
+ return t;
+ }
+
+ @Override
+ public int length() {
+ return len + 4;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeInt(this.len);
+ out.write(this.data, 0, this.len);
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ this.len = in.readInt();
+ in.readFully(this.data, 0, this.len);
+ }
+
+ @Override
+ public int hashCode() {
+ final byte[] copy = new byte[this.len];
+ System.arraycopy(this.data, 0, copy, 0, this.len);
+ return Arrays.hashCode(copy);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof ByteSubArrayType) {
+ ByteSubArrayType other = (ByteSubArrayType) obj;
+ if (this.len == other.len) {
+ for (int i = 0; i < this.len; i++) {
+ if (this.data[i] != other.data[i]) {
+ return false;
+ }
+ }
+ return true;
+ } else {
+ return false;
+ }
+ } else {
+ return false;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteType.java
new file mode 100644
index 0000000..52abdcb
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class ByteType implements SerializationTestType {
+
+ private byte value;
+
+ public ByteType() {
+ this.value = (byte) 0;
+ }
+
+ private ByteType(byte value) {
+ this.value = value;
+ }
+
+ @Override
+ public ByteType getRandom(Random rnd) {
+ return new ByteType((byte) rnd.nextInt(256));
+ }
+
+ @Override
+ public int length() {
+ return 1;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeByte(this.value);
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ this.value = in.readByte();
+ }
+
+ @Override
+ public int hashCode() {
+ return this.value;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof ByteType) {
+ ByteType other = (ByteType) obj;
+ return this.value == other.value;
+ } else {
+ return false;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/CharType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/CharType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/CharType.java
new file mode 100644
index 0000000..25df737
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/CharType.java
@@ -0,0 +1,68 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class CharType implements SerializationTestType {
+
+ private char value;
+
+ public CharType() {
+ this.value = 0;
+ }
+
+ private CharType(char value) {
+ this.value = value;
+ }
+
+ @Override
+ public CharType getRandom(Random rnd) {
+ return new CharType((char) rnd.nextInt(10000));
+ }
+
+ @Override
+ public int length() {
+ return 2;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeChar(this.value);
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ this.value = in.readChar();
+ }
+
+ @Override
+ public int hashCode() {
+ return this.value;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof CharType) {
+ CharType other = (CharType) obj;
+ return this.value == other.value;
+ } else {
+ return false;
+ }
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/DoubleType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/DoubleType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/DoubleType.java
new file mode 100644
index 0000000..1046e75
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/DoubleType.java
@@ -0,0 +1,68 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class DoubleType implements SerializationTestType {
+
+ private double value;
+
+ public DoubleType() {
+ this.value = 0;
+ }
+
+ private DoubleType(double value) {
+ this.value = value;
+ }
+
+ @Override
+ public DoubleType getRandom(Random rnd) {
+ return new DoubleType(rnd.nextDouble());
+ }
+
+ @Override
+ public int length() {
+ return 8;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeDouble(this.value);
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ this.value = in.readDouble();
+ }
+
+ @Override
+ public int hashCode() {
+ final long l = Double.doubleToLongBits(this.value);
+ return (int) (l ^ l >>> 32);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof DoubleType) {
+ DoubleType other = (DoubleType) obj;
+ return Double.doubleToLongBits(this.value) == Double.doubleToLongBits(other.value);
+ } else {
+ return false;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/FloatType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/FloatType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/FloatType.java
new file mode 100644
index 0000000..4a50873
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/FloatType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class FloatType implements SerializationTestType {
+
+ private float value;
+
+ public FloatType() {
+ this.value = 0;
+ }
+
+ private FloatType(float value) {
+ this.value = value;
+ }
+
+ @Override
+ public FloatType getRandom(Random rnd) {
+ return new FloatType(rnd.nextFloat());
+ }
+
+ @Override
+ public int length() {
+ return 4;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeFloat(this.value);
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ this.value = in.readFloat();
+ }
+
+ @Override
+ public int hashCode() {
+ return Float.floatToIntBits(this.value);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof FloatType) {
+ FloatType other = (FloatType) obj;
+ return Float.floatToIntBits(this.value) == Float.floatToIntBits(other.value);
+ } else {
+ return false;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/IntType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/IntType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/IntType.java
new file mode 100644
index 0000000..50a3546
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/IntType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class IntType implements SerializationTestType {
+
+ private int value;
+
+ public IntType() {
+ this.value = 0;
+ }
+
+ public IntType(int value) {
+ this.value = value;
+ }
+
+ @Override
+ public IntType getRandom(Random rnd) {
+ return new IntType(rnd.nextInt());
+ }
+
+ @Override
+ public int length() {
+ return 4;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeInt(this.value);
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ this.value = in.readInt();
+ }
+
+ @Override
+ public int hashCode() {
+ return this.value;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof IntType) {
+ IntType other = (IntType) obj;
+ return this.value == other.value;
+ } else {
+ return false;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/LongType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/LongType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/LongType.java
new file mode 100644
index 0000000..1402fb5
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/LongType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class LongType implements SerializationTestType {
+
+ private long value;
+
+ public LongType() {
+ this.value = 0;
+ }
+
+ private LongType(long value) {
+ this.value = value;
+ }
+
+ @Override
+ public LongType getRandom(Random rnd) {
+ return new LongType(rnd.nextLong());
+ }
+
+ @Override
+ public int length() {
+ return 8;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeLong(this.value);
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ this.value = in.readLong();
+ }
+
+ @Override
+ public int hashCode() {
+ return (int) (this.value ^ this.value >>> 32);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof LongType) {
+ LongType other = (LongType) obj;
+ return this.value == other.value;
+ } else {
+ return false;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestType.java
new file mode 100644
index 0000000..a827b07
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestType.java
@@ -0,0 +1,26 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.util.Random;
+
+import eu.stratosphere.core.io.IOReadableWritable;
+
+public interface SerializationTestType extends IOReadableWritable {
+
+ public SerializationTestType getRandom(Random rnd);
+
+ public int length();
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestTypeFactory.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestTypeFactory.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestTypeFactory.java
new file mode 100644
index 0000000..127a0ec
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/SerializationTestTypeFactory.java
@@ -0,0 +1,40 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+public enum SerializationTestTypeFactory {
+ BOOLEAN(new BooleanType()),
+ BYTE_ARRAY(new ByteArrayType()),
+ BYTE_SUB_ARRAY(new ByteSubArrayType()),
+ BYTE(new ByteType()),
+ CHAR(new CharType()),
+ DOUBLE(new DoubleType()),
+ FLOAT(new FloatType()),
+ INT(new IntType()),
+ LONG(new LongType()),
+ SHORT(new ShortType()),
+ UNSIGNED_BYTE(new UnsignedByteType()),
+ UNSIGNED_SHORT(new UnsignedShortType()),
+ STRING(new AsciiStringType());
+
+ private final SerializationTestType factory;
+
+ SerializationTestTypeFactory(SerializationTestType type) {
+ this.factory = type;
+ }
+
+ public SerializationTestType factory() {
+ return this.factory;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ShortType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ShortType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ShortType.java
new file mode 100644
index 0000000..7711e88
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ShortType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class ShortType implements SerializationTestType {
+
+ private short value;
+
+ public ShortType() {
+ this.value = (short) 0;
+ }
+
+ private ShortType(short value) {
+ this.value = value;
+ }
+
+ @Override
+ public ShortType getRandom(Random rnd) {
+ return new ShortType((short) rnd.nextInt(65536));
+ }
+
+ @Override
+ public int length() {
+ return 2;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeShort(this.value);
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ this.value = in.readShort();
+ }
+
+ @Override
+ public int hashCode() {
+ return this.value;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof ShortType) {
+ ShortType other = (ShortType) obj;
+ return this.value == other.value;
+ } else {
+ return false;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedByteType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedByteType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedByteType.java
new file mode 100644
index 0000000..9a1f1fb
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedByteType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class UnsignedByteType implements SerializationTestType {
+
+ private int value;
+
+ public UnsignedByteType() {
+ this.value = 0;
+ }
+
+ private UnsignedByteType(int value) {
+ this.value = value;
+ }
+
+ @Override
+ public UnsignedByteType getRandom(Random rnd) {
+ return new UnsignedByteType(rnd.nextInt(128) + 128);
+ }
+
+ @Override
+ public int length() {
+ return 1;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeByte(this.value);
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ this.value = in.readUnsignedByte();
+ }
+
+ @Override
+ public int hashCode() {
+ return this.value;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof UnsignedByteType) {
+ UnsignedByteType other = (UnsignedByteType) obj;
+ return this.value == other.value;
+ } else {
+ return false;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedShortType.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedShortType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedShortType.java
new file mode 100644
index 0000000..ac80ef7
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedShortType.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Random;
+
+public class UnsignedShortType implements SerializationTestType {
+
+ private int value;
+
+ public UnsignedShortType() {
+ this.value = 0;
+ }
+
+ private UnsignedShortType(int value) {
+ this.value = value;
+ }
+
+ @Override
+ public UnsignedShortType getRandom(Random rnd) {
+ return new UnsignedShortType(rnd.nextInt(32768) + 32768);
+ }
+
+ @Override
+ public int length() {
+ return 2;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ out.writeShort(this.value);
+ }
+
+ @Override
+ public void read(DataInput in) throws IOException {
+ this.value = in.readUnsignedShort();
+ }
+
+ @Override
+ public int hashCode() {
+ return this.value;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof UnsignedShortType) {
+ UnsignedShortType other = (UnsignedShortType) obj;
+ return this.value == other.value;
+ } else {
+ return false;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/Util.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/Util.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/Util.java
new file mode 100644
index 0000000..ef14f7f
--- /dev/null
+++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/Util.java
@@ -0,0 +1,90 @@
+/***********************************************************************************************************************
+ * 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.serialization.types;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Random;
+
+public class Util {
+
+ private static final long SEED = 64871654635745873L;
+
+ private static Random random = new Random(SEED);
+
+ public static SerializationTestType randomRecord(SerializationTestTypeFactory type) {
+ return type.factory().getRandom(Util.random);
+ }
+
+ public static MockRecords randomRecords(final int numElements, final SerializationTestTypeFactory type) {
+
+ return new MockRecords(numElements) {
+ @Override
+ protected SerializationTestType getRecord() {
+ return type.factory().getRandom(Util.random);
+ }
+ };
+ }
+
+ public static MockRecords randomRecords(final int numElements) {
+
+ return new MockRecords(numElements) {
+ @Override
+ protected SerializationTestType getRecord() {
+ // select random test type factory
+ SerializationTestTypeFactory[] types = SerializationTestTypeFactory.values();
+ int i = Util.random.nextInt(types.length);
+
+ return types[i].factory().getRandom(Util.random);
+ }
+ };
+ }
+
+ // -----------------------------------------------------------------------------------------------------------------
+ public abstract static class MockRecords implements Iterable<SerializationTestType> {
+
+ private int numRecords;
+
+ public MockRecords(int numRecords) {
+ this.numRecords = numRecords;
+ }
+
+ @Override
+ public Iterator<SerializationTestType> iterator() {
+ return new Iterator<SerializationTestType>() {
+ @Override
+ public boolean hasNext() {
+ return numRecords > 0;
+ }
+
+ @Override
+ public SerializationTestType next() {
+ if (numRecords > 0) {
+ numRecords--;
+
+ return getRecord();
+ }
+
+ throw new NoSuchElementException();
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+ };
+ }
+
+ abstract protected SerializationTestType getRecord();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2db78a8d/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/BroadcastVarsNepheleITCase.java
----------------------------------------------------------------------
diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/BroadcastVarsNepheleITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/BroadcastVarsNepheleITCase.java
index db843dd..b80810b 100644
--- a/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/BroadcastVarsNepheleITCase.java
+++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/broadcastvars/BroadcastVarsNepheleITCase.java
@@ -21,6 +21,8 @@ import java.util.regex.Matcher;
import java.util.regex.Pattern;
import eu.stratosphere.test.util.RecordAPITestBase;
+import eu.stratosphere.nephele.jobgraph.DistributionPattern;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import org.junit.Assert;
import eu.stratosphere.api.common.operators.util.UserCodeClassWrapper;
@@ -31,8 +33,6 @@ import eu.stratosphere.api.java.record.io.CsvInputFormat;
import eu.stratosphere.api.java.record.io.CsvOutputFormat;
import eu.stratosphere.configuration.Configuration;
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.JobGraphDefinitionException;
import eu.stratosphere.nephele.jobgraph.JobInputVertex;