diff --git a/flink-addons/flink-streaming/pom.xml b/flink-addons/flink-streaming/pom.xml
index 25d69b057f6bb6e1edbedfb91c483e1ce77b1bf2..50fe3bd90adb66e7e50dca9a873d9c2c3a0e6654 100644
--- a/flink-addons/flink-streaming/pom.xml
+++ b/flink-addons/flink-streaming/pom.xml
@@ -12,7 +12,7 @@
jar
- 0.5
+ 0.6-SNAPSHOT
UTF-8
UTF-8
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/BatchReduceInvokable.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/BatchReduceInvokable.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/BatchReduceInvokable.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/BatchReduceInvokable.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/DataStream.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/DataStream.java
similarity index 97%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/DataStream.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/DataStream.java
index cc590e2e8c5bfeb8f26a647f31aefb60ddbcdaa1..6244d5788adecb4afae27091ae57977e1a4ab8f6 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/DataStream.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/DataStream.java
@@ -127,7 +127,7 @@ public class DataStream {
}
return returnStream;
}
-
+
/**
* Connecting DataStream outputs with each other. The streams connected
* using this operator will be transformed simultaneously. It creates a
@@ -137,21 +137,13 @@ public class DataStream {
* The DataStream to connect output with.
* @return The connected DataStream.
*/
- public DataStream connectWith(DataStream... streams) {
+ public DataStream connectWith(DataStream stream) {
DataStream returnStream = copy();
-
- for(DataStream stream:streams){
- addConnection(returnStream, stream);
- }
- return returnStream;
- }
-
- public DataStream addConnection(DataStream returnStream, DataStream stream){
+
returnStream.connectIDs.addAll(stream.connectIDs);
returnStream.ctypes.addAll(stream.ctypes);
returnStream.cparams.addAll(stream.cparams);
returnStream.batchSizes.addAll(stream.batchSizes);
-
return returnStream;
}
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/FileSourceFunction.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/FileSourceFunction.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/FileSourceFunction.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/FileSourceFunction.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/FileStreamFunction.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/FileStreamFunction.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/FileStreamFunction.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/FileStreamFunction.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/FilterInvokable.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/FilterInvokable.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/FilterInvokable.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/FilterInvokable.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/FlatMapInvokable.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/FlatMapInvokable.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/FlatMapInvokable.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/FlatMapInvokable.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/JobGraphBuilder.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/JobGraphBuilder.java
similarity index 98%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/JobGraphBuilder.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/JobGraphBuilder.java
index 181d8263ff922141f4a1da59b3f4ff1c6810411b..37becf9289de8d9f53e002e53aa1a3e877bedbc4 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/JobGraphBuilder.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/JobGraphBuilder.java
@@ -28,8 +28,6 @@ import org.apache.commons.logging.LogFactory;
import eu.stratosphere.api.java.tuple.Tuple;
import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.nephele.io.ChannelSelector;
-import eu.stratosphere.nephele.io.channels.ChannelType;
import eu.stratosphere.nephele.jobgraph.AbstractJobVertex;
import eu.stratosphere.nephele.jobgraph.JobGraph;
import eu.stratosphere.nephele.jobgraph.JobGraphDefinitionException;
@@ -37,6 +35,8 @@ import eu.stratosphere.nephele.jobgraph.JobInputVertex;
import eu.stratosphere.nephele.jobgraph.JobOutputVertex;
import eu.stratosphere.nephele.jobgraph.JobTaskVertex;
import eu.stratosphere.pact.runtime.task.util.TaskConfig;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
+import eu.stratosphere.runtime.io.channels.ChannelType;
import eu.stratosphere.streaming.api.invokable.UserSinkInvokable;
import eu.stratosphere.streaming.api.invokable.UserSourceInvokable;
import eu.stratosphere.streaming.api.invokable.UserTaskInvokable;
@@ -127,7 +127,7 @@ public class JobGraphBuilder {
final JobInputVertex source = new JobInputVertex(sourceName, jobGraph);
- source.setInputClass(StreamSource.class);
+ source.setInvokableClass(StreamSource.class);
setComponent(sourceName, source, InvokableObject, operatorName, serializedFunction,
parallelism, subtasksPerInstance);
@@ -158,7 +158,7 @@ public class JobGraphBuilder {
String operatorName, byte[] serializedFunction, int parallelism, int subtasksPerInstance) {
final JobTaskVertex task = new JobTaskVertex(taskName, jobGraph);
- task.setTaskClass(StreamTask.class);
+ task.setInvokableClass(StreamTask.class);
setComponent(taskName, task, TaskInvokableObject, operatorName, serializedFunction,
parallelism, subtasksPerInstance);
@@ -187,7 +187,7 @@ public class JobGraphBuilder {
String operatorName, byte[] serializedFunction, int parallelism, int subtasksPerInstance) {
final JobOutputVertex sink = new JobOutputVertex(sinkName, jobGraph);
- sink.setOutputClass(StreamSink.class);
+ sink.setInvokableClass(StreamSink.class);
setComponent(sinkName, sink, InvokableObject, operatorName, serializedFunction,
parallelism, subtasksPerInstance);
@@ -220,7 +220,8 @@ public class JobGraphBuilder {
int parallelism, int subtasksPerInstance) {
component.setNumberOfSubtasks(parallelism);
- component.setNumberOfSubtasksPerInstance(subtasksPerInstance);
+ // TODO remove all NumberOfSubtasks setting
+// component.setNumberOfSubtasksPerInstance(subtasksPerInstance);
if (parallelism > maxParallelism) {
maxParallelism = parallelism;
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/MapInvokable.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/MapInvokable.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/MapInvokable.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/MapInvokable.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/SinkFunction.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/SinkFunction.java
similarity index 86%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/SinkFunction.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/SinkFunction.java
index a2880a44fc231bf199fd8b416c5247e9cae2e8af..ff30915b7343443468969384eb55168b8680f020 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/SinkFunction.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/SinkFunction.java
@@ -17,10 +17,9 @@ package eu.stratosphere.streaming.api;
import java.io.Serializable;
-import eu.stratosphere.api.common.functions.AbstractFunction;
import eu.stratosphere.api.java.tuple.Tuple;
-public abstract class SinkFunction extends AbstractFunction implements Serializable {
+public abstract class SinkFunction implements Serializable {
private static final long serialVersionUID = 1L;
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/SinkInvokable.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/SinkInvokable.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/SinkInvokable.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/SinkInvokable.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/SourceFunction.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/SourceFunction.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/SourceFunction.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/SourceFunction.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/StreamCollector.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/StreamCollector.java
similarity index 95%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/StreamCollector.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/StreamCollector.java
index 192c54ff1a3161839f4e9b27d5acc43aaed7c524..8f7efec529be327e952b434dac24c0c4be1a015f 100755
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/StreamCollector.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/StreamCollector.java
@@ -16,8 +16,8 @@
package eu.stratosphere.streaming.api;
import eu.stratosphere.api.java.tuple.Tuple;
-import eu.stratosphere.nephele.io.RecordWriter;
import eu.stratosphere.pact.runtime.plugable.SerializationDelegate;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.streaming.api.streamrecord.ArrayStreamRecord;
import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
import eu.stratosphere.util.Collector;
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/StreamCollectorManager.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/StreamCollectorManager.java
similarity index 98%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/StreamCollectorManager.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/StreamCollectorManager.java
index 63e127edfb0e28a43a061734a727bc4f0ce2bf71..8a873eb683d1e94817c6624375ab7fa5d9f106de 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/StreamCollectorManager.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/StreamCollectorManager.java
@@ -19,8 +19,8 @@ import java.util.ArrayList;
import java.util.List;
import eu.stratosphere.api.java.tuple.Tuple;
-import eu.stratosphere.nephele.io.RecordWriter;
import eu.stratosphere.pact.runtime.plugable.SerializationDelegate;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
import eu.stratosphere.util.Collector;
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/StreamExecutionEnvironment.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/StreamExecutionEnvironment.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/StreamExecutionEnvironment.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/StreamExecutionEnvironment.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/DefaultSinkInvokable.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/DefaultSinkInvokable.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/DefaultSinkInvokable.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/DefaultSinkInvokable.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/DefaultSourceInvokable.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/DefaultSourceInvokable.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/DefaultSourceInvokable.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/DefaultSourceInvokable.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/DefaultTaskInvokable.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/DefaultTaskInvokable.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/DefaultTaskInvokable.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/DefaultTaskInvokable.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/StreamComponent.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/StreamComponent.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/StreamComponent.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/StreamComponent.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/StreamRecordInvokable.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/StreamRecordInvokable.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/StreamRecordInvokable.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/StreamRecordInvokable.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/UserSinkInvokable.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/UserSinkInvokable.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/UserSinkInvokable.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/UserSinkInvokable.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/UserSourceInvokable.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/UserSourceInvokable.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/UserSourceInvokable.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/UserSourceInvokable.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/UserTaskInvokable.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/UserTaskInvokable.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/invokable/UserTaskInvokable.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/invokable/UserTaskInvokable.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentException.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentException.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentException.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentException.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentHelper.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentHelper.java
similarity index 69%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentHelper.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentHelper.java
index 84742bc32d8a8e37cbb79e6d98226ad6f6319acc..ed47da7359dd11885325fdf052da4aa356e28c89 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentHelper.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentHelper.java
@@ -37,19 +37,18 @@ import eu.stratosphere.api.java.typeutils.runtime.TupleSerializer;
import eu.stratosphere.configuration.Configuration;
import eu.stratosphere.nephele.event.task.AbstractTaskEvent;
import eu.stratosphere.nephele.event.task.EventListener;
-import eu.stratosphere.nephele.io.AbstractRecordReader;
-import eu.stratosphere.nephele.io.ChannelSelector;
-import eu.stratosphere.nephele.io.MutableRecordReader;
-import eu.stratosphere.nephele.io.RecordWriter;
import eu.stratosphere.nephele.template.AbstractInvokable;
import eu.stratosphere.pact.runtime.plugable.DeserializationDelegate;
import eu.stratosphere.pact.runtime.plugable.SerializationDelegate;
+import eu.stratosphere.runtime.io.api.AbstractRecordReader;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
+import eu.stratosphere.runtime.io.api.MutableRecordReader;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.streaming.api.SinkFunction;
import eu.stratosphere.streaming.api.StreamCollectorManager;
import eu.stratosphere.streaming.api.invokable.DefaultSinkInvokable;
import eu.stratosphere.streaming.api.invokable.DefaultSourceInvokable;
import eu.stratosphere.streaming.api.invokable.DefaultTaskInvokable;
-import eu.stratosphere.streaming.api.invokable.StreamComponent;
import eu.stratosphere.streaming.api.invokable.StreamRecordInvokable;
import eu.stratosphere.streaming.api.invokable.UserSinkInvokable;
import eu.stratosphere.streaming.api.invokable.UserSourceInvokable;
@@ -78,13 +77,13 @@ public final class StreamComponentHelper {
private SerializationDelegate outSerializationDelegate = null;
public Collector collector;
- private List batchSizesNotPartitioned = new ArrayList();
- private List batchSizesPartitioned = new ArrayList();
- private List numOfOutputsPartitioned = new ArrayList();
+ private List batchsizes_s = new ArrayList();
+ private List batchsizes_f = new ArrayList();
+ private List numOfOutputs_f = new ArrayList();
private int keyPosition = 0;
- private List> outputsNotPartitioned = new ArrayList>();
- private List> outputsPartitioned = new ArrayList>();
+ private List> outputs_s = new ArrayList>();
+ private List> outputs_f = new ArrayList>();
public static int newComponent() {
numComponents++;
@@ -118,35 +117,47 @@ public final class StreamComponentHelper {
public Collector setCollector(Configuration taskConfiguration, int id,
List> outputs) {
+ int batchSize = taskConfiguration.getInteger("batchSize", 1);
+
long batchTimeout = taskConfiguration.getLong("batchTimeout", 1000);
+ // collector = new StreamCollector(batchSize, batchTimeout, id,
+ // outSerializationDelegate, outputs);
- collector = new StreamCollectorManager(batchSizesNotPartitioned,
- batchSizesPartitioned, numOfOutputsPartitioned, keyPosition, batchTimeout, id,
- outSerializationDelegate, outputsPartitioned, outputsNotPartitioned);
+ collector = new StreamCollectorManager(batchsizes_s, batchsizes_f, numOfOutputs_f,
+ keyPosition, batchTimeout, id, outSerializationDelegate, outputs_f, outputs_s);
return collector;
}
+ // TODO add type parameters to avoid redundant code
+ @SuppressWarnings({ "rawtypes", "unchecked" })
public void setSerializers(Configuration taskConfiguration) {
byte[] operatorBytes = taskConfiguration.getBytes("operator", null);
String operatorName = taskConfiguration.getString("operatorName", "");
- Object function = null;
try {
ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(operatorBytes));
- function = in.readObject();
+ Object function = in.readObject();
if (operatorName.equals("flatMap")) {
- setSerializerDeserializer(function, FlatMapFunction.class);
+ setSerializer(function, FlatMapFunction.class);
} else if (operatorName.equals("map")) {
- setSerializerDeserializer(function, MapFunction.class);
+ setSerializer(function, MapFunction.class);
} else if (operatorName.equals("batchReduce")) {
- setSerializerDeserializer(function, GroupReduceFunction.class);
+ setSerializer(function, GroupReduceFunction.class);
} else if (operatorName.equals("filter")) {
- setSerializerDeserializer(function, FilterFunction.class);
+ setSerializer(function, FilterFunction.class);
} else if (operatorName.equals("sink")) {
- setDeserializer(function, SinkFunction.class);
+ inTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(SinkFunction.class,
+ function.getClass(), 0, null, null);
+
+ inTupleSerializer = inTupleTypeInfo.createSerializer();
+ inDeserializationDelegate = new DeserializationDelegate(inTupleSerializer);
} else if (operatorName.equals("source")) {
- setSerializer(function, UserSourceInvokable.class, 0);
+ outTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(
+ UserSourceInvokable.class, function.getClass(), 0, null, null);
+
+ outTupleSerializer = outTupleTypeInfo.createSerializer();
+ outSerializationDelegate = new SerializationDelegate(outTupleSerializer);
} else if (operatorName.equals("elements")) {
outTupleTypeInfo = new TupleTypeInfo(TypeExtractor.getForObject(function));
@@ -157,43 +168,25 @@ public final class StreamComponentHelper {
}
} catch (Exception e) {
- throw new StreamComponentException("Nonsupported object (named " + operatorName
- + ") passed as operator");
- }
- }
+ throw new StreamComponentException("Nonsupported object passed as operator");
- private void setSerializerDeserializer(Object function, Class extends AbstractFunction> clazz) {
- setDeserializer(function, clazz);
- setSerializer(function, clazz, 1);
+ }
}
- @SuppressWarnings({ "unchecked", "rawtypes" })
- private void setDeserializer(Object function, Class extends AbstractFunction> clazz) {
+ private void setSerializer(Object function, Class extends AbstractFunction> clazz) {
inTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz, function.getClass(),
0, null, null);
inTupleSerializer = inTupleTypeInfo.createSerializer();
inDeserializationDelegate = new DeserializationDelegate(inTupleSerializer);
- }
- @SuppressWarnings({ "rawtypes", "unchecked" })
- private void setSerializer(Object function, Class> clazz, int typeParameter) {
outTupleTypeInfo = (TupleTypeInfo) TypeExtractor.createTypeInfo(clazz, function.getClass(),
- typeParameter, null, null);
+ 1, null, null);
outTupleSerializer = outTupleTypeInfo.createSerializer();
outSerializationDelegate = new SerializationDelegate(outTupleSerializer);
}
- public void setSinkSerializer() {
- if (outSerializationDelegate != null) {
- inTupleTypeInfo = outTupleTypeInfo;
-
- inTupleSerializer = inTupleTypeInfo.createSerializer();
- inDeserializationDelegate = new DeserializationDelegate(inTupleSerializer);
- }
- }
-
public AbstractRecordReader getConfigInputs(T taskBase, Configuration taskConfiguration)
throws StreamComponentException {
int numberOfInputs = taskConfiguration.getInteger("numberOfInputs", 0);
@@ -240,40 +233,33 @@ public final class StreamComponentHelper {
if (taskBase instanceof StreamTask) {
outputs.add(new RecordWriter((StreamTask) taskBase,
- StreamRecord.class, outputPartitioner));
+ outputPartitioner));
} else if (taskBase instanceof StreamSource) {
outputs.add(new RecordWriter((StreamSource) taskBase,
- StreamRecord.class, outputPartitioner));
+ outputPartitioner));
} else {
throw new StreamComponentException("Nonsupported object passed to setConfigOutputs");
}
- if (outputsPartitioned.size() < batchSizesPartitioned.size()) {
- outputsPartitioned.add(outputs.get(i));
+ if (outputs_f.size() < batchsizes_f.size()) {
+ outputs_f.add(outputs.get(i));
} else {
- outputsNotPartitioned.add(outputs.get(i));
+ outputs_s.add(outputs.get(i));
}
}
}
- /**
- * Reads and creates a StreamComponent from the config.
- *
- * @param userFunctionClass
- * Class of the invokable function
- * @param config
- * Configuration object
- * @return The StreamComponent object
- */
- private StreamComponent getInvokable(Class extends StreamComponent> userFunctionClass,
- Configuration config) {
- StreamComponent userFunction = null;
-
- byte[] userFunctionSerialized = config.getBytes("serializedudf", null);
+ public UserSinkInvokable getSinkInvokable(Configuration taskConfiguration) {
+
+ Class extends UserSinkInvokable> userFunctionClass = taskConfiguration.getClass(
+ "userfunction", DefaultSinkInvokable.class, UserSinkInvokable.class);
+ UserSinkInvokable userFunction = null;
+
+ byte[] userFunctionSerialized = taskConfiguration.getBytes("serializedudf", null);
try {
ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(
userFunctionSerialized));
- userFunction = (StreamComponent) ois.readObject();
+ userFunction = (UserSinkInvokable) ois.readObject();
} catch (Exception e) {
if (log.isErrorEnabled()) {
log.error("Cannot instanciate user function: " + userFunctionClass.getSimpleName());
@@ -283,30 +269,58 @@ public final class StreamComponentHelper {
return userFunction;
}
- @SuppressWarnings("rawtypes")
- public UserSinkInvokable getSinkInvokable(Configuration config) {
- Class extends UserSinkInvokable> userFunctionClass = config.getClass("userfunction",
- DefaultSinkInvokable.class, UserSinkInvokable.class);
- return (UserSinkInvokable) getInvokable(userFunctionClass, config);
- }
-
// TODO consider logging stack trace!
- @SuppressWarnings("rawtypes")
- public UserTaskInvokable getTaskInvokable(Configuration config) {
+ @SuppressWarnings("unchecked")
+ public UserTaskInvokable getTaskInvokable(Configuration taskConfiguration) {
// Default value is a TaskInvokable even if it was called from a source
- Class extends UserTaskInvokable> userFunctionClass = config.getClass("userfunction",
- DefaultTaskInvokable.class, UserTaskInvokable.class);
- return (UserTaskInvokable) getInvokable(userFunctionClass, config);
+ Class extends UserTaskInvokable> userFunctionClass = taskConfiguration.getClass(
+ "userfunction", DefaultTaskInvokable.class, UserTaskInvokable.class);
+ UserTaskInvokable userFunction = null;
+
+ byte[] userFunctionSerialized = taskConfiguration.getBytes("serializedudf", null);
+
+ try {
+ ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(
+ userFunctionSerialized));
+ userFunction = (UserTaskInvokable) ois.readObject();
+ // userFunction.declareOutputs(outputs, instanceID, name,
+ // recordBuffer,
+ // faultToleranceType);
+ } catch (Exception e) {
+ if (log.isErrorEnabled()) {
+
+ log.error("Cannot instanciate user function: " + userFunctionClass.getSimpleName());
+ }
+ }
+
+ return userFunction;
}
- @SuppressWarnings("rawtypes")
- public UserSourceInvokable getSourceInvokable(Configuration config) {
-
+ public UserSourceInvokable getSourceInvokable(Configuration taskConfiguration) {
+
// Default value is a TaskInvokable even if it was called from a source
- Class extends UserSourceInvokable> userFunctionClass = config.getClass("userfunction",
- DefaultSourceInvokable.class, UserSourceInvokable.class);
- return (UserSourceInvokable) getInvokable(userFunctionClass, config);
+ Class extends UserSourceInvokable> userFunctionClass = taskConfiguration.getClass(
+ "userfunction", DefaultSourceInvokable.class, UserSourceInvokable.class);
+ UserSourceInvokable userFunction = null;
+
+ byte[] userFunctionSerialized = taskConfiguration.getBytes("serializedudf", null);
+
+ try {
+ ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(
+ userFunctionSerialized));
+ userFunction = (UserSourceInvokable) ois.readObject();
+ // userFunction.declareOutputs(outputs, instanceID, name,
+ // recordBuffer,
+ // faultToleranceType);
+ } catch (Exception e) {
+ if (log.isErrorEnabled()) {
+
+ log.error("Cannot instanciate user function: " + userFunctionClass.getSimpleName());
+ }
+ }
+
+ return userFunction;
}
// TODO find a better solution for this
@@ -326,40 +340,47 @@ public final class StreamComponentHelper {
}
}
- private void setPartitioner(Configuration config, int numberOfOutputs,
+ private void setPartitioner(Configuration taskConfiguration, int nrOutput,
List> partitioners) {
- Class extends ChannelSelector> partitioner = config.getClass(
- "partitionerClass_" + numberOfOutputs, DefaultPartitioner.class,
- ChannelSelector.class);
+ Class extends ChannelSelector> partitioner = taskConfiguration.getClass(
+ "partitionerClass_" + nrOutput, DefaultPartitioner.class, ChannelSelector.class);
- Integer batchSize = config.getInteger("batchSize_" + numberOfOutputs, 1);
+ Integer batchSize = taskConfiguration.getInteger("batchSize_" + nrOutput, 1);
try {
if (partitioner.equals(FieldsPartitioner.class)) {
- batchSizesPartitioned.add(batchSize);
- numOfOutputsPartitioned.add(config
- .getInteger("numOfOutputs_" + numberOfOutputs, -1));
+ batchsizes_f.add(batchSize);
+ numOfOutputs_f.add(taskConfiguration.getInteger("numOfOutputs_" + nrOutput, -1));
// TODO:force one partitioning field
- keyPosition = config.getInteger("partitionerIntParam_" + numberOfOutputs, 1);
+ keyPosition = taskConfiguration.getInteger("partitionerIntParam_" + nrOutput, 1);
partitioners.add(partitioner.getConstructor(int.class).newInstance(keyPosition));
} else {
- batchSizesNotPartitioned.add(batchSize);
+ batchsizes_s.add(batchSize);
partitioners.add(partitioner.newInstance());
}
if (log.isTraceEnabled()) {
- log.trace("Partitioner set: " + partitioner.getSimpleName() + " with "
- + numberOfOutputs + " outputs");
+ log.trace("Partitioner set: " + partitioner.getSimpleName() + " with " + nrOutput
+ + " outputs");
}
} catch (Exception e) {
if (log.isErrorEnabled()) {
log.error("Error while setting partitioner: " + partitioner.getSimpleName()
- + " with " + numberOfOutputs + " outputs", e);
+ + " with " + nrOutput + " outputs", e);
}
}
}
+ public void setSinkSerializer() {
+ if (outSerializationDelegate != null) {
+ inTupleTypeInfo = outTupleTypeInfo;
+
+ inTupleSerializer = inTupleTypeInfo.createSerializer();
+ inDeserializationDelegate = new DeserializationDelegate(inTupleSerializer);
+ }
+ }
+
public void invokeRecords(StreamRecordInvokable userFunction, AbstractRecordReader inputs)
throws Exception {
if (inputs instanceof UnionStreamRecordReader) {
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamInvokableComponent.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamInvokableComponent.java
similarity index 98%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamInvokableComponent.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamInvokableComponent.java
index 3d46ddf72fc90b66fa50f0abb273b9d710fbb97e..e9b217e7cb3f95755faec50d1753add68f1b1b81 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamInvokableComponent.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamInvokableComponent.java
@@ -22,7 +22,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import eu.stratosphere.api.java.tuple.Tuple;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
import eu.stratosphere.streaming.faulttolerance.FaultToleranceType;
import eu.stratosphere.streaming.faulttolerance.FaultToleranceUtil;
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamRecordReader.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamRecordReader.java
similarity index 56%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamRecordReader.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamRecordReader.java
index 19f1e6ec83355e89c11dba7775ba2abfda1b90b8..04d6e99f0d62ce477ed1707cce380ceb7a64f143 100755
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamRecordReader.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamRecordReader.java
@@ -1,6 +1,5 @@
/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu)
+ * 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
@@ -10,7 +9,6 @@
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
- *
**********************************************************************************************************************/
package eu.stratosphere.streaming.api.streamcomponent;
@@ -19,126 +17,100 @@ import java.io.IOException;
import eu.stratosphere.api.java.tuple.Tuple;
import eu.stratosphere.api.java.typeutils.runtime.TupleSerializer;
-import eu.stratosphere.nephele.io.AbstractSingleGateRecordReader;
-import eu.stratosphere.nephele.io.InputChannelResult;
-import eu.stratosphere.nephele.io.MutableRecordDeserializerFactory;
-import eu.stratosphere.nephele.io.Reader;
-import eu.stratosphere.nephele.template.AbstractOutputTask;
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.nephele.template.AbstractInvokable;
import eu.stratosphere.pact.runtime.plugable.DeserializationDelegate;
+import eu.stratosphere.runtime.io.api.AbstractSingleGateRecordReader;
+import eu.stratosphere.runtime.io.api.Reader;
+import eu.stratosphere.runtime.io.gates.InputChannelResult;
import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
/**
- * A record writer connects an input gate to an application. It allows the
- * application query for incoming records and read them from input gate.
+ * 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 The type of the record that can be read from this record reader.
*/
-public class StreamRecordReader extends AbstractSingleGateRecordReader implements
- Reader {
-
+public class StreamRecordReader extends AbstractSingleGateRecordReader implements Reader {
+
private final Class extends StreamRecord> recordType;
private DeserializationDelegate deserializationDelegate;
private TupleSerializer tupleSerializer;
-
/**
* Stores the last read record.
*/
private StreamRecord lookahead;
/**
- * Stores if more no more records will be received from the assigned input
- * gate.
+ * 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.
+ * 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 StreamRecordReader(AbstractTask taskBase,
- Class extends StreamRecord> recordType,
- DeserializationDelegate deserializationDelegate,
- TupleSerializer tupleSerializer) {
- // super(taskBase, MutableRecordDeserializerFactory. get(), 0);
- super(taskBase,MutableRecordDeserializerFactory.get(), 0);
- this.recordType = recordType;
- this.deserializationDelegate = deserializationDelegate;
- this.tupleSerializer = tupleSerializer;
- }
-
- /**
- * 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.
+ * The application that instantiated the record reader.
* @param recordType
- * The class of records that can be read from the record reader.
+ * The class of records that can be read from the record reader.
*/
- public StreamRecordReader(AbstractOutputTask outputBase,
- Class extends StreamRecord> recordType,
+ public StreamRecordReader(AbstractInvokable taskBase, Class extends StreamRecord> recordType,
DeserializationDelegate deserializationDelegate,
TupleSerializer tupleSerializer) {
- // super(outputBase, MutableRecordDeserializerFactory. get(), 0);
- super(outputBase,MutableRecordDeserializerFactory.get(), 0);
+ super(taskBase);
this.recordType = recordType;
this.deserializationDelegate = deserializationDelegate;
this.tupleSerializer = tupleSerializer;
}
-
+
// --------------------------------------------------------------------------------------------
-
+
/**
- * 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.
+ * 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 true
it at least one more record can be read from the
- * associated input gate, otherwise false
+ * @return true
it at least one more record can be read from the associated input gate, otherwise
+ * false
*/
@Override
- public boolean hasNext() throws IOException, InterruptedException {
+ public boolean hasNext() throws IOException, InterruptedException{
if (this.lookahead != null) {
return true;
} else {
if (this.noMoreRecordsWillFollow) {
return false;
}
-
+
StreamRecord record = instantiateRecordType();
record.setDeseralizationDelegate(deserializationDelegate, tupleSerializer);
-
+
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())
+ 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;
- 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
+
+ default:
+ ; // fall through the loop
}
}
}
@@ -149,8 +121,7 @@ public class StreamRecordReader extends AbstractSingleGateRecordReader {
+public class StreamSource extends DataSourceTask {
private static final Log log = LogFactory.getLog(StreamSource.class);
@@ -55,16 +55,6 @@ public class StreamSource extends AbstractInputTask {
sourceInstanceID = numSources;
}
- @Override
- public DummyIS[] computeInputSplits(int requestedMinNumber) throws Exception {
- return null;
- }
-
- @Override
- public Class getInputSplitType() {
- return null;
- }
-
@Override
public void registerInputOutput() {
Configuration taskConfiguration = getTaskConfiguration();
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamTask.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamTask.java
similarity index 93%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamTask.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamTask.java
index a48e322e6abed30d3dd59250d0ef06bad2f959fa..e83609c420bc05f7bdd5f74ac4dd150197dd6c95 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamTask.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamTask.java
@@ -22,16 +22,16 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import eu.stratosphere.configuration.Configuration;
-import eu.stratosphere.nephele.io.AbstractRecordReader;
-import eu.stratosphere.nephele.io.ChannelSelector;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractTask;
+import eu.stratosphere.pact.runtime.task.RegularPactTask;
+import eu.stratosphere.runtime.io.api.AbstractRecordReader;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.streaming.api.invokable.UserTaskInvokable;
import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
import eu.stratosphere.streaming.faulttolerance.FaultToleranceType;
import eu.stratosphere.streaming.faulttolerance.FaultToleranceUtil;
-public class StreamTask extends AbstractTask {
+public class StreamTask extends RegularPactTask {
private static final Log log = LogFactory.getLog(StreamTask.class);
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamWindowTask.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamWindowTask.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamWindowTask.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/StreamWindowTask.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/UnionStreamRecordReader.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/UnionStreamRecordReader.java
similarity index 94%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/UnionStreamRecordReader.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/UnionStreamRecordReader.java
index c408dae8f6639ac8ba08439802c3e0f593a42c68..75693f09eeb2561bd4af263738dc36e53802323c 100755
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamcomponent/UnionStreamRecordReader.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamcomponent/UnionStreamRecordReader.java
@@ -19,10 +19,10 @@ import java.io.IOException;
import eu.stratosphere.api.java.tuple.Tuple;
import eu.stratosphere.api.java.typeutils.runtime.TupleSerializer;
-import eu.stratosphere.nephele.io.AbstractUnionRecordReader;
-import eu.stratosphere.nephele.io.MutableRecordReader;
-import eu.stratosphere.nephele.io.Reader;
import eu.stratosphere.pact.runtime.plugable.DeserializationDelegate;
+import eu.stratosphere.runtime.io.api.AbstractUnionRecordReader;
+import eu.stratosphere.runtime.io.api.MutableRecordReader;
+import eu.stratosphere.runtime.io.api.Reader;
import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
public final class UnionStreamRecordReader extends AbstractUnionRecordReader
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/ArrayStreamRecord.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/ArrayStreamRecord.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/ArrayStreamRecord.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/ArrayStreamRecord.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/FieldTypeMismatchException.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/FieldTypeMismatchException.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/FieldTypeMismatchException.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/FieldTypeMismatchException.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/ListStreamRecord.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/ListStreamRecord.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/ListStreamRecord.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/ListStreamRecord.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/NoSuchFieldException.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/NoSuchFieldException.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/NoSuchFieldException.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/NoSuchFieldException.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/NoSuchTupleException.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/NoSuchTupleException.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/NoSuchTupleException.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/NoSuchTupleException.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/StreamRecord.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/StreamRecord.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/StreamRecord.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/StreamRecord.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/StreamRecordException.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/StreamRecordException.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/StreamRecordException.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/StreamRecordException.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/StreamRecordGeneric.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/StreamRecordGeneric.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/StreamRecordGeneric.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/StreamRecordGeneric.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/TupleSizeMismatchException.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/TupleSizeMismatchException.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/TupleSizeMismatchException.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/TupleSizeMismatchException.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/UID.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/UID.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/api/streamrecord/UID.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/api/streamrecord/UID.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/DummyIS.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/DummyIS.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/DummyIS.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/DummyIS.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/basictopology/BasicTopology.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/basictopology/BasicTopology.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/basictopology/BasicTopology.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/basictopology/BasicTopology.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/cellinfo/CellInfoLocal.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/cellinfo/CellInfoLocal.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/cellinfo/CellInfoLocal.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/cellinfo/CellInfoLocal.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/cellinfo/IWorkerEngine.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/cellinfo/IWorkerEngine.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/cellinfo/IWorkerEngine.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/cellinfo/IWorkerEngine.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/cellinfo/Util.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/cellinfo/Util.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/cellinfo/Util.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/cellinfo/Util.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/cellinfo/WorkerEngineBin.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/cellinfo/WorkerEngineBin.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/cellinfo/WorkerEngineBin.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/cellinfo/WorkerEngineBin.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/cellinfo/WorkerEngineExact.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/cellinfo/WorkerEngineExact.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/cellinfo/WorkerEngineExact.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/cellinfo/WorkerEngineExact.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringLocal.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringLocal.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringLocal.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringLocal.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringMap.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringMap.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringMap.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringMap.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringReduce.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringReduce.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringReduce.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringReduce.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSink.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSink.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSink.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSink.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/collaborativefilter/CollaborativeFilteringSource.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansLocal.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansLocal.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansLocal.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansLocal.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansMap.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansMap.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansMap.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansMap.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansReduce.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansReduce.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansReduce.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansReduce.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansSink.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansSink.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansSink.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansSink.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansSource.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansSource.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansSource.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/kmeans/KMeansSource.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankLocal.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankLocal.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankLocal.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankLocal.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankMap.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankMap.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankMap.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankMap.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankReduce.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankReduce.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankReduce.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankReduce.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankSink.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankSink.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankSink.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankSink.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankSource.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankSource.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankSource.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/pagerank/PageRankSource.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPLocal.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPLocal.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPLocal.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPLocal.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPMap.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPMap.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPMap.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPMap.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPReduce.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPReduce.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPReduce.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPReduce.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPSink.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPSink.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPSink.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPSink.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPSource.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPSource.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPSource.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/iterative/sssp/SSSPSource.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/join/JoinLocal.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/join/JoinLocal.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/join/JoinLocal.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/join/JoinLocal.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/join/JoinSink.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/join/JoinSink.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/join/JoinSink.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/join/JoinSink.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/join/JoinSourceOne.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/join/JoinSourceOne.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/join/JoinSourceOne.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/join/JoinSourceOne.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/join/JoinSourceTwo.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/join/JoinSourceTwo.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/join/JoinSourceTwo.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/join/JoinSourceTwo.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/join/JoinTask.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/join/JoinTask.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/join/JoinTask.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/join/JoinTask.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/ml/IncrementalLearningSkeleton.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/ml/IncrementalLearningSkeleton.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/ml/IncrementalLearningSkeleton.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/ml/IncrementalLearningSkeleton.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/ml/IncrementalOLS.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/ml/IncrementalOLS.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/ml/IncrementalOLS.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/ml/IncrementalOLS.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinLocal.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinLocal.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinLocal.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinLocal.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinSourceOne.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinSourceOne.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinSourceOne.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinSourceOne.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinSourceTwo.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinSourceTwo.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinSourceTwo.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinSourceTwo.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinTask.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinTask.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinTask.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/join/WindowJoinTask.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumAggregate.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumAggregate.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumAggregate.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumAggregate.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumLocal.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumLocal.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumLocal.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumLocal.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumMultiple.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumMultiple.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumMultiple.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumMultiple.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumSink.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumSink.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumSink.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumSink.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumSource.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumSource.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumSource.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/sum/WindowSumSource.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountCounter.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountCounter.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountCounter.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountCounter.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountLocal.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountLocal.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountLocal.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountLocal.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountSink.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountSink.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountSink.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountSink.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountSource.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountSource.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountSource.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountSource.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountSplitter.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountSplitter.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountSplitter.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/window/wordcount/WindowWordCountSplitter.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountCounter.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountCounter.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountCounter.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountCounter.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountLocal.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountLocal.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountLocal.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountLocal.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountSplitter.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountSplitter.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountSplitter.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/examples/wordcount/WordCountSplitter.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/AckEvent.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/AckEvent.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/AckEvent.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/AckEvent.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/AckEventListener.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/AckEventListener.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/AckEventListener.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/AckEventListener.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/AtLeastOnceFaultToleranceBuffer.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/AtLeastOnceFaultToleranceBuffer.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/AtLeastOnceFaultToleranceBuffer.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/AtLeastOnceFaultToleranceBuffer.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/ExactlyOnceFaultToleranceBuffer.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/ExactlyOnceFaultToleranceBuffer.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/ExactlyOnceFaultToleranceBuffer.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/ExactlyOnceFaultToleranceBuffer.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/FailEvent.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/FailEvent.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/FailEvent.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/FailEvent.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/FailEventListener.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/FailEventListener.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/FailEventListener.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/FailEventListener.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceBuffer.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceBuffer.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceBuffer.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceBuffer.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceType.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceType.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceType.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceType.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceUtil.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceUtil.java
similarity index 99%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceUtil.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceUtil.java
index 673e923d5a1f7504f1592a377ea66a1eaab429ed..8c6de1ff70f241436c7c31bdb7c8263193232319 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceUtil.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceUtil.java
@@ -20,7 +20,7 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
import eu.stratosphere.streaming.api.streamrecord.UID;
import eu.stratosphere.streaming.util.PerformanceCounter;
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/index/BTreeIndex.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/index/BTreeIndex.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/index/BTreeIndex.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/index/BTreeIndex.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/index/IndexPair.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/index/IndexPair.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/index/IndexPair.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/index/IndexPair.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/main/java/eu/stratosphere/streaming/addons/kafka/KafkaProducer.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/kafka/KafkaProducer.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/main/java/eu/stratosphere/streaming/addons/kafka/KafkaProducer.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/kafka/KafkaProducer.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/main/java/eu/stratosphere/streaming/addons/kafka/KafkaSource.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/kafka/KafkaSource.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/main/java/eu/stratosphere/streaming/addons/kafka/KafkaSource.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/kafka/KafkaSource.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/main/java/eu/stratosphere/streaming/addons/kafka/KafkaTopology.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/kafka/KafkaTopology.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/main/java/eu/stratosphere/streaming/addons/kafka/KafkaTopology.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/kafka/KafkaTopology.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/partitioner/BroadcastPartitioner.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/partitioner/BroadcastPartitioner.java
similarity index 95%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/partitioner/BroadcastPartitioner.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/partitioner/BroadcastPartitioner.java
index b539cd0224ec0e146fa797e61d4e79226605535b..ab22fa072077b436fd06a76ebf9d00d173c89104 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/partitioner/BroadcastPartitioner.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/partitioner/BroadcastPartitioner.java
@@ -15,7 +15,7 @@
package eu.stratosphere.streaming.partitioner;
-import eu.stratosphere.nephele.io.ChannelSelector;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
public class BroadcastPartitioner implements ChannelSelector {
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/partitioner/DefaultPartitioner.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/partitioner/DefaultPartitioner.java
similarity index 95%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/partitioner/DefaultPartitioner.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/partitioner/DefaultPartitioner.java
index 7b0843ea7e0161851f0da19a9dcb0b04676a555b..e66687f24b0cb1d9a45c1457c10fc1f4ae395d4a 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/partitioner/DefaultPartitioner.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/partitioner/DefaultPartitioner.java
@@ -15,7 +15,7 @@
package eu.stratosphere.streaming.partitioner;
-import eu.stratosphere.nephele.io.ChannelSelector;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
public class DefaultPartitioner implements ChannelSelector {
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/partitioner/FieldsPartitioner.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/partitioner/FieldsPartitioner.java
similarity index 96%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/partitioner/FieldsPartitioner.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/partitioner/FieldsPartitioner.java
index c1abdcba007e1422d25bcd09bc70c4eb76821cda..b71527abec3f2e19fe79e288cde8aef7a29cd79c 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/partitioner/FieldsPartitioner.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/partitioner/FieldsPartitioner.java
@@ -15,7 +15,7 @@
package eu.stratosphere.streaming.partitioner;
-import eu.stratosphere.nephele.io.ChannelSelector;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
//Grouping by a key
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/partitioner/GlobalPartitioner.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/partitioner/GlobalPartitioner.java
similarity index 95%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/partitioner/GlobalPartitioner.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/partitioner/GlobalPartitioner.java
index 5fae51b1c72238d30261a0dd79f317b7a05ea771..dc5bc6d28a5671cf12636dd6bf1d9b49e2321b58 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/partitioner/GlobalPartitioner.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/partitioner/GlobalPartitioner.java
@@ -15,7 +15,7 @@
package eu.stratosphere.streaming.partitioner;
-import eu.stratosphere.nephele.io.ChannelSelector;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
//Group to the partitioner with the lowest id
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/partitioner/ShufflePartitioner.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/partitioner/ShufflePartitioner.java
similarity index 96%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/partitioner/ShufflePartitioner.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/partitioner/ShufflePartitioner.java
index 56930fdaf2be58e4ac9434e9dca4ea1ebe772717..580162d1a9b7a083c50df4fc5c720d8bbaeccbe0 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/partitioner/ShufflePartitioner.java
+++ b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/partitioner/ShufflePartitioner.java
@@ -17,7 +17,7 @@ package eu.stratosphere.streaming.partitioner;
import java.util.Random;
-import eu.stratosphere.nephele.io.ChannelSelector;
+import eu.stratosphere.runtime.io.api.ChannelSelector;
import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
//Randomly group, to distribute equally
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/main/java/eu/stratosphere/streaming/addons/performance/WordCountPerformanceLocal.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/performance/WordCountPerformanceLocal.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/main/java/eu/stratosphere/streaming/addons/performance/WordCountPerformanceLocal.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/performance/WordCountPerformanceLocal.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/main/java/eu/stratosphere/streaming/addons/performance/WordCountPerformanceSplitter.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/performance/WordCountPerformanceSplitter.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/main/java/eu/stratosphere/streaming/addons/performance/WordCountPerformanceSplitter.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/performance/WordCountPerformanceSplitter.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/main/java/eu/stratosphere/streaming/addons/rabbitmq/RMQSource.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/rabbitmq/RMQSource.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/main/java/eu/stratosphere/streaming/addons/rabbitmq/RMQSource.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/rabbitmq/RMQSource.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/main/java/eu/stratosphere/streaming/addons/rabbitmq/RMQTopology.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/rabbitmq/RMQTopology.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/main/java/eu/stratosphere/streaming/addons/rabbitmq/RMQTopology.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/rabbitmq/RMQTopology.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/GraphState.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/GraphState.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/GraphState.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/GraphState.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/MutableTableState.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/MutableTableState.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/MutableTableState.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/MutableTableState.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/MutableTableStateIterator.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/MutableTableStateIterator.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/MutableTableStateIterator.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/MutableTableStateIterator.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/SlidingWindowState.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/SlidingWindowState.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/SlidingWindowState.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/SlidingWindowState.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/SlidingWindowStateIterator.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/SlidingWindowStateIterator.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/SlidingWindowStateIterator.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/SlidingWindowStateIterator.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/TableState.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/TableState.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/TableState.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/TableState.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/TableStateIterator.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/TableStateIterator.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/TableStateIterator.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/TableStateIterator.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/manager/StateCheckpointer.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/manager/StateCheckpointer.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/manager/StateCheckpointer.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/manager/StateCheckpointer.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/manager/StateRestorer.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/manager/StateRestorer.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/state/manager/StateRestorer.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/state/manager/StateRestorer.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/util/ClusterUtil.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/util/ClusterUtil.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/util/ClusterUtil.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/util/ClusterUtil.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/util/LogUtils.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/util/LogUtils.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/util/LogUtils.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/util/LogUtils.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/util/PerformanceCounter.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/util/PerformanceCounter.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/util/PerformanceCounter.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/util/PerformanceCounter.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/util/PerformanceTimer.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/util/PerformanceTimer.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/util/PerformanceTimer.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/util/PerformanceTimer.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/util/PerformanceTracker.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/util/PerformanceTracker.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/util/PerformanceTracker.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/util/PerformanceTracker.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/util/TestDataUtil.java b/flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/util/TestDataUtil.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/main/java/eu/stratosphere/streaming/util/TestDataUtil.java
rename to flink-addons/flink-streaming/src/main/java/eu/stratosphere/streaming/util/TestDataUtil.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/BatchReduceTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/BatchReduceTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/BatchReduceTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/BatchReduceTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/BatchTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/BatchTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/BatchTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/BatchTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/FlatMapTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/FlatMapTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/FlatMapTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/FlatMapTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/MapTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/MapTest.java
similarity index 73%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/MapTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/MapTest.java
index 5b181ffa30f997858dd2c8ba11df64eb60a4092c..c92e97c563e4e84c839e7c1bfba68735ff747ef3 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/MapTest.java
+++ b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/MapTest.java
@@ -26,7 +26,6 @@ import org.junit.Test;
import eu.stratosphere.api.java.functions.MapFunction;
import eu.stratosphere.api.java.tuple.Tuple1;
-import eu.stratosphere.api.java.tuple.Tuple3;
import eu.stratosphere.util.Collector;
public class MapTest {
@@ -40,36 +39,6 @@ public class MapTest {
}
}
}
-
- public static final class MySource1 extends SourceFunction> {
-
- @Override
- public void invoke(Collector> collector) throws Exception {
- for (int i = 0; i < 5; i++) {
- collector.collect(new Tuple1(i));
- }
- }
- }
-
- public static final class MySource2 extends SourceFunction> {
-
- @Override
- public void invoke(Collector> collector) throws Exception {
- for (int i = 5; i < 10; i++) {
- collector.collect(new Tuple1(i));
- }
- }
- }
-
- public static final class MySource3 extends SourceFunction> {
-
- @Override
- public void invoke(Collector> collector) throws Exception {
- for (int i = 10; i < 15; i++) {
- collector.collect(new Tuple1(i));
- }
- }
- }
public static final class MyMap extends MapFunction, Tuple1> {
@@ -79,15 +48,6 @@ public class MapTest {
return new Tuple1(value.f0 * value.f0);
}
}
-
- public static final class MyJoinMap extends MapFunction, Tuple1> {
-
- @Override
- public Tuple1 map(Tuple1 value) throws Exception {
- joinSetResult.add(value.f0);
- return new Tuple1(value.f0);
- }
- }
public static final class MyFieldsMap extends MapFunction, Tuple1> {
@@ -162,13 +122,6 @@ public class MapTest {
graphResult++;
}
}
-
- public static final class JoinSink extends SinkFunction> {
-
- @Override
- public void invoke(Tuple1 tuple) {
- }
- }
private static Set expected = new HashSet();
private static Set result = new HashSet();
@@ -185,9 +138,6 @@ public class MapTest {
private static Set fromCollectionSet = new HashSet();
private static List fromCollectionFields = new ArrayList();
private static Set fromCollectionDiffFieldsSet = new HashSet();
- private static Set singleJoinSetExpected = new HashSet();
- private static Set multipleJoinSetExpected = new HashSet();
- private static Set joinSetResult = new HashSet();
private static void fillExpectedList() {
for (int i = 0; i < 10; i++) {
@@ -219,19 +169,6 @@ public class MapTest {
}
}
}
-
- private static void fillSingleJoinSet() {
- for (int i = 0; i < 10; i++) {
- singleJoinSetExpected.add(i);
- }
- }
-
- private static void fillMultipleJoinSet() {
- for (int i = 0; i < 15; i++) {
- multipleJoinSetExpected.add(i);
- }
- }
-
@Test
public void mapTest() throws Exception {
@@ -350,51 +287,5 @@ public class MapTest {
// }
//
// }
-
- @Test
- public void singleConnectWithTest() throws Exception {
-
- StreamExecutionEnvironment env = new StreamExecutionEnvironment();
-
- DataStream> source1 = env.addSource(new MySource1(),
- 1);
-
- DataStream> source2 = env
- .addSource(new MySource2(), 1)
- .connectWith(source1)
- .partitionBy(0)
- .map(new MyJoinMap(), 1)
- .addSink(new JoinSink());
-
- env.execute();
-
- fillSingleJoinSet();
-
- assertEquals(singleJoinSetExpected, joinSetResult);
- }
-
- @Test
- public void multipleConnectWithTest() throws Exception {
-
- StreamExecutionEnvironment env = new StreamExecutionEnvironment();
-
- DataStream> source1 = env.addSource(new MySource1(),
- 1);
-
- DataStream> source2 = env.addSource(new MySource2(),
- 1);
- DataStream> source3 = env
- .addSource(new MySource3(), 1)
- .connectWith(source1, source2)
- .partitionBy(0)
- .map(new MyJoinMap(), 1)
- .addSink(new JoinSink());
-
- env.execute();
-
- fillMultipleJoinSet();
-
- assertEquals(multipleJoinSetExpected, joinSetResult);
- }
}
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/PrintTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/PrintTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/PrintTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/PrintTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/StreamCollectorManagerTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/StreamCollector2Test.java
similarity index 92%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/StreamCollectorManagerTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/StreamCollector2Test.java
index c8ca0e84aca44a7bb9286ab04632b5a462933b50..91f5eb888623bce67fd3f63f9f34f44a83529db7 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/StreamCollectorManagerTest.java
+++ b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/StreamCollector2Test.java
@@ -15,7 +15,7 @@
package eu.stratosphere.streaming.api;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
import java.util.ArrayList;
import java.util.List;
@@ -24,12 +24,12 @@ import org.junit.Test;
import eu.stratosphere.api.java.tuple.Tuple;
import eu.stratosphere.api.java.tuple.Tuple1;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.streaming.api.streamcomponent.MockRecordWriter;
import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
import eu.stratosphere.streaming.util.MockRecordWriterFactory;
-public class StreamCollectorManagerTest {
+public class StreamCollector2Test {
StreamCollectorManager collector;
@@ -55,7 +55,9 @@ public class StreamCollectorManagerTest {
fOut.add(rw2);
collector = new StreamCollectorManager(batchSizesOfNotPartitioned, batchSizesOfPartitioned, parallelismOfOutput, keyPosition, batchTimeout, channelID, null, fOut,fOut);
+
Tuple1 t = new Tuple1();
+ StreamCollector sc1 = new StreamCollector(1, batchTimeout, channelID, null);
t.f0 = 0;
collector.collect(t);
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/StreamCollectorTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/StreamCollectorTest.java
similarity index 82%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/StreamCollectorTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/StreamCollectorTest.java
index 812b21abb9ef3daa6c1d0eda27deab183b0497b5..ada3d1e90c349454691fc411fe0c04e80162be77 100755
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/StreamCollectorTest.java
+++ b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/StreamCollectorTest.java
@@ -27,13 +27,13 @@ public class StreamCollectorTest {
@Test
public void testStreamCollector() {
- StreamCollector> collector = new StreamCollector>(10, 1000, 0, null);
+ StreamCollector collector = new StreamCollector(10, 1000, 0, null);
assertEquals(10, collector.batchSize);
}
@Test
public void testCollect() {
- StreamCollector> collector = new StreamCollector>(2, 1000, 0, null);
+ StreamCollector collector = new StreamCollector(2, 1000, 0, null);
collector.collect(new Tuple1(3));
collector.collect(new Tuple1(4));
collector.collect(new Tuple1(5));
@@ -43,7 +43,8 @@ public class StreamCollectorTest {
@Test
public void testBatchSize() throws InterruptedException {
- StreamCollector> collector = new StreamCollector>(3, 100, 0, null);
+ System.out.println("---------------");
+ StreamCollector collector = new StreamCollector(3, 100, 0, null);
collector.collect(new Tuple1(0));
collector.collect(new Tuple1(0));
collector.collect(new Tuple1(0));
@@ -51,13 +52,14 @@ public class StreamCollectorTest {
Thread.sleep(200);
collector.collect(new Tuple1(2));
collector.collect(new Tuple1(3));
+ System.out.println("---------------");
}
@Test
public void recordWriter() {
MockRecordWriter recWriter = MockRecordWriterFactory.create();
- StreamCollector> collector = new StreamCollector>(2, 1000, 0, null, recWriter);
+ StreamCollector collector = new StreamCollector(2, 1000, 0, null, recWriter);
collector.collect(new Tuple1(3));
collector.collect(new Tuple1(4));
collector.collect(new Tuple1(5));
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/TypeExtractTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/TypeExtractTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/TypeExtractTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/TypeExtractTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamcomponent/MockRecordWriter.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamcomponent/MockRecordWriter.java
similarity index 85%
rename from flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamcomponent/MockRecordWriter.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamcomponent/MockRecordWriter.java
index e25d7d91b8a781341ca4e14d1142fb3e92b3f953..973a3a69a64f2a577d3b75077c7274ef4a4b61ad 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamcomponent/MockRecordWriter.java
+++ b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamcomponent/MockRecordWriter.java
@@ -16,16 +16,16 @@ package eu.stratosphere.streaming.api.streamcomponent;
import java.util.ArrayList;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.nephele.template.AbstractInputTask;
+import eu.stratosphere.pact.runtime.task.DataSourceTask;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
public class MockRecordWriter extends RecordWriter {
public ArrayList emittedRecords;
- public MockRecordWriter(AbstractInputTask> inputBase, Class outputClass) {
- super(inputBase, outputClass);
+ public MockRecordWriter(DataSourceTask> inputBase, Class outputClass) {
+ super(inputBase);
}
public boolean initList() {
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/streamrecord/ArrayStreamRecordTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamrecord/ArrayStreamRecordTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/streamrecord/ArrayStreamRecordTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamrecord/ArrayStreamRecordTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/streamrecord/ListStreamRecordTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamrecord/ListStreamRecordTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/streamrecord/ListStreamRecordTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamrecord/ListStreamRecordTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/streamrecord/MyGeneric.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamrecord/MyGeneric.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/streamrecord/MyGeneric.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamrecord/MyGeneric.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/streamrecord/MyGeneric2.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamrecord/MyGeneric2.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/streamrecord/MyGeneric2.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamrecord/MyGeneric2.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/streamrecord/UUIDTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamrecord/UUIDTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/streamrecord/UUIDTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/api/streamrecord/UUIDTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/faulttolerance/AtLeastOnceBufferTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/faulttolerance/AtLeastOnceBufferTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/faulttolerance/AtLeastOnceBufferTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/faulttolerance/AtLeastOnceBufferTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/faulttolerance/ExactlyOnceBufferTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/faulttolerance/ExactlyOnceBufferTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/faulttolerance/ExactlyOnceBufferTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/faulttolerance/ExactlyOnceBufferTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceTypeTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceTypeTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceTypeTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceTypeTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceUtilTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceUtilTest.java
similarity index 97%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceUtilTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceUtilTest.java
index daa725f4e79387d8375bfb90c0b365ce0945a881..ca9b6322192be785cd154ed2357b09a88c9849ce 100644
--- a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceUtilTest.java
+++ b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceUtilTest.java
@@ -21,7 +21,7 @@ import java.util.List;
import org.junit.Before;
import org.junit.Test;
-import eu.stratosphere.nephele.io.RecordWriter;
+import eu.stratosphere.runtime.io.api.RecordWriter;
import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
public class FaultToleranceUtilTest {
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/index/BTreeIndexTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/index/BTreeIndexTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/index/BTreeIndexTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/index/BTreeIndexTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/partitioner/BroadcastPartitionerTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/partitioner/BroadcastPartitionerTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/partitioner/BroadcastPartitionerTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/partitioner/BroadcastPartitionerTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/partitioner/DefaultPartitionerTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/partitioner/DefaultPartitionerTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/partitioner/DefaultPartitionerTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/partitioner/DefaultPartitionerTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/partitioner/FieldsPartitionerTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/partitioner/FieldsPartitionerTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/partitioner/FieldsPartitionerTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/partitioner/FieldsPartitionerTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/partitioner/GlobalPartitionerTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/partitioner/GlobalPartitionerTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/partitioner/GlobalPartitionerTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/partitioner/GlobalPartitionerTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/partitioner/ShufflePartitionerTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/partitioner/ShufflePartitionerTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/partitioner/ShufflePartitionerTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/partitioner/ShufflePartitionerTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/state/InternalStateTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/state/InternalStateTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/state/InternalStateTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/state/InternalStateTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/util/MockRecordWriterFactory.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/util/MockRecordWriterFactory.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/util/MockRecordWriterFactory.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/util/MockRecordWriterFactory.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/util/PerformanceTrackerTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/util/PerformanceTrackerTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/util/PerformanceTrackerTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/util/PerformanceTrackerTest.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/util/TestDataUtilTest.java b/flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/util/TestDataUtilTest.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/util/TestDataUtilTest.java
rename to flink-addons/flink-streaming/src/test/java/eu/stratosphere/streaming/util/TestDataUtilTest.java
diff --git a/flink-addons/flink-streaming/src/test/resources/ClusterConfigs/localConfig1TM.prop b/flink-addons/flink-streaming/src/test/resources/ClusterConfigs/localConfig1TM.prop
new file mode 100644
index 0000000000000000000000000000000000000000..a1e4ad927d8cafb081e4310ecbdd38fa5eed397b
--- /dev/null
+++ b/flink-addons/flink-streaming/src/test/resources/ClusterConfigs/localConfig1TM.prop
@@ -0,0 +1,4 @@
+ClusterProvider#clusterId=local1TM
+LocalClusterProvider#numTaskTrackers=1
+ClusterProvider#clusterProviderType=LocalClusterProvider
+FilesystemProvider#filesystemType=local_fs
\ No newline at end of file
diff --git a/flink-addons/flink-streaming/src/test/resources/ClusterConfigs/localConfig4TM.prop b/flink-addons/flink-streaming/src/test/resources/ClusterConfigs/localConfig4TM.prop
new file mode 100644
index 0000000000000000000000000000000000000000..ea7e670ff612d16b8fffecee5bfb317064395c42
--- /dev/null
+++ b/flink-addons/flink-streaming/src/test/resources/ClusterConfigs/localConfig4TM.prop
@@ -0,0 +1,4 @@
+ClusterProvider#clusterId=local4TM
+LocalClusterProvider#numTaskTrackers=4
+ClusterProvider#clusterProviderType=LocalClusterProvider
+FilesystemProvider#filesystemType=mini_hdfs
\ No newline at end of file
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/resources/Performance/PerformanceTracker.py b/flink-addons/flink-streaming/src/test/resources/Performance/PerformanceTracker.py
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/resources/Performance/PerformanceTracker.py
rename to flink-addons/flink-streaming/src/test/resources/Performance/PerformanceTracker.py
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/resources/Performance/WordCountTopology.java b/flink-addons/flink-streaming/src/test/resources/Performance/WordCountTopology.java
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/resources/Performance/WordCountTopology.java
rename to flink-addons/flink-streaming/src/test/resources/Performance/WordCountTopology.java
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/resources/Performance/copy-files.sh b/flink-addons/flink-streaming/src/test/resources/Performance/copy-files.sh
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/resources/Performance/copy-files.sh
rename to flink-addons/flink-streaming/src/test/resources/Performance/copy-files.sh
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/resources/Performance/copy-logged-files.sh b/flink-addons/flink-streaming/src/test/resources/Performance/copy-logged-files.sh
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/resources/Performance/copy-logged-files.sh
rename to flink-addons/flink-streaming/src/test/resources/Performance/copy-logged-files.sh
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/resources/Performance/remove-files.sh b/flink-addons/flink-streaming/src/test/resources/Performance/remove-files.sh
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/resources/Performance/remove-files.sh
rename to flink-addons/flink-streaming/src/test/resources/Performance/remove-files.sh
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/resources/Performance/run-test.sh b/flink-addons/flink-streaming/src/test/resources/Performance/run-test.sh
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/resources/Performance/run-test.sh
rename to flink-addons/flink-streaming/src/test/resources/Performance/run-test.sh
diff --git a/flink-addons/flink-streaming/src/test/resources/TestConfigs/eu.stratosphere.pact.test.contracts.prop b/flink-addons/flink-streaming/src/test/resources/TestConfigs/eu.stratosphere.pact.test.contracts.prop
new file mode 100644
index 0000000000000000000000000000000000000000..b21d989af384066ac382fc865488c6fac167c06f
--- /dev/null
+++ b/flink-addons/flink-streaming/src/test/resources/TestConfigs/eu.stratosphere.pact.test.contracts.prop
@@ -0,0 +1,5 @@
+CoGroupITCase=local1TM,local4TM
+CrossITCase=local1TM
+MapITCase=local1TM,local4TM
+MatchITCase=local1TM
+ReduceITCase=local1TM
\ No newline at end of file
diff --git a/flink-addons/flink-streaming/src/test/resources/TestConfigs/eu.stratosphere.pact.test.pactPrograms.prop b/flink-addons/flink-streaming/src/test/resources/TestConfigs/eu.stratosphere.pact.test.pactPrograms.prop
new file mode 100644
index 0000000000000000000000000000000000000000..fd81262923645d3f3e9a8f098c46edd4aca14ff3
--- /dev/null
+++ b/flink-addons/flink-streaming/src/test/resources/TestConfigs/eu.stratosphere.pact.test.pactPrograms.prop
@@ -0,0 +1,5 @@
+TPCHQuery3ITCase=local1TM,local4TM
+TPCHQuery4ITCase=local1TM,local4TM
+TPCHQuery9ITCase=local1TM,local4TM
+TPCHQuery10ITCase=local1TM,local4TM
+GlobalSortingITCase=local1TM,local4TM
\ No newline at end of file
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/test/resources/testdata_checksum/ASTopology.data.md5 b/flink-addons/flink-streaming/src/test/resources/testdata_checksum/ASTopology.data.md5
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/test/resources/testdata_checksum/ASTopology.data.md5
rename to flink-addons/flink-streaming/src/test/resources/testdata_checksum/ASTopology.data.md5
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/test/resources/testdata_checksum/MovieLens100k.data.md5 b/flink-addons/flink-streaming/src/test/resources/testdata_checksum/MovieLens100k.data.md5
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/test/resources/testdata_checksum/MovieLens100k.data.md5
rename to flink-addons/flink-streaming/src/test/resources/testdata_checksum/MovieLens100k.data.md5
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/test/resources/testdata_checksum/hamlet.txt.md5 b/flink-addons/flink-streaming/src/test/resources/testdata_checksum/hamlet.txt.md5
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/test/resources/testdata_checksum/hamlet.txt.md5
rename to flink-addons/flink-streaming/src/test/resources/testdata_checksum/hamlet.txt.md5
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/src/test/resources/testdata_checksum/terainput.txt.md5 b/flink-addons/flink-streaming/src/test/resources/testdata_checksum/terainput.txt.md5
similarity index 100%
rename from flink-addons/flink-streaming/stratosphere-streaming-examples/src/test/resources/testdata_checksum/terainput.txt.md5
rename to flink-addons/flink-streaming/src/test/resources/testdata_checksum/terainput.txt.md5
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/pom.xml b/flink-addons/flink-streaming/stratosphere-streaming-addons/pom.xml
deleted file mode 100644
index 25d69b057f6bb6e1edbedfb91c483e1ce77b1bf2..0000000000000000000000000000000000000000
--- a/flink-addons/flink-streaming/stratosphere-streaming-addons/pom.xml
+++ /dev/null
@@ -1,248 +0,0 @@
-
-
-
- 4.0.0
-
- eu.stratosphere
- 0.2-SNAPSHOT
- stratosphere-streaming
- stratosphere-streaming
-
- jar
-
-
- 0.5
- UTF-8
- UTF-8
-
-
-
-
- dms-repo
- https://dms.sztaki.hu/maven-public
- true
- true
-
-
-
-
-
- eu.stratosphere
- stratosphere-core
- ${stratosphere.version}
-
-
- eu.stratosphere
- stratosphere-tests
- ${stratosphere.version}
-
-
- eu.stratosphere
- stratosphere-compiler
- ${stratosphere.version}
-
-
- eu.stratosphere
- stratosphere-runtime
- ${stratosphere.version}
-
-
- eu.stratosphere
- stratosphere-clients
- ${stratosphere.version}
-
-
- eu.stratosphere
- stratosphere-java
- ${stratosphere.version}
-
-
- junit
- junit
- 4.7
-
-
- org.apache.commons
- commons-lang3
- 3.3.2
-
-
- commons-logging
- commons-logging
- 1.1.1
- jar
- compile
-
-
- log4j
- log4j
- 1.2.16
-
-
- com.rabbitmq
- amqp-client
- 3.3.1
-
-
- org.apache.kafka
- kafka_2.10
- 0.8.0
-
-
- org.jblas
- jblas
- 1.2.3
-
-
- org.mockito
- mockito-all
- 1.8.5
- test
-
-
-
-
-
-
-
- org.apache.maven.plugins
- maven-compiler-plugin
- 3.1
-
- 1.6
- 1.6
-
-
-
-
- org.apache.maven.plugins
- maven-javadoc-plugin
- 2.9.1
-
- true
-
-
-
- attach-javadocs
-
- jar
-
-
-
-
-
- maven-failsafe-plugin
- 2.6
-
-
-
- integration-test
- verify
-
-
-
-
- -Xmx1024m
-
-
-
- org.apache.maven.plugins
- maven-eclipse-plugin
- 2.8
-
-
-
- org.eclipse.jdt.launching.JRE_CONTAINER
-
-
- true
- true
-
-
-
- org.apache.maven.plugins
- maven-source-plugin
- 2.2.1
-
-
- attach-sources
-
- jar
-
-
-
-
-
- org.apache.maven.plugins
- maven-jar-plugin
- 2.2
-
-
-
- test-jar
-
-
-
-
-
- org.apache.maven.plugins
- maven-surefire-plugin
- 2.7
-
-
- WARN
-
-
- **/*TestBase*.class
-
- once
- -Xmx1024m
-
-
-
- org.apache.rat
- apache-rat-plugin
- 0.10
-
-
- verify
-
- check
-
-
-
-
- false
- 0
-
-
- **/.*
-
- **/resources/**
-
- **/stratosphere-bin/conf/slaves
-
- README.md
- CHANGELOG
- CONTRIBUTORS
-
- **/pom.xml
-
- **/target/**
-
-
-
-
- org.apache.maven.plugins
- maven-assembly-plugin
- 2.4
-
-
- jar-with-dependencies
-
-
-
-
-
-
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/MapTest.java b/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/MapTest.java
deleted file mode 100644
index 5b181ffa30f997858dd2c8ba11df64eb60a4092c..0000000000000000000000000000000000000000
--- a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/MapTest.java
+++ /dev/null
@@ -1,400 +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.streaming.api;
-
-import static org.junit.Assert.*;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.junit.Test;
-
-import eu.stratosphere.api.java.functions.MapFunction;
-import eu.stratosphere.api.java.tuple.Tuple1;
-import eu.stratosphere.api.java.tuple.Tuple3;
-import eu.stratosphere.util.Collector;
-
-public class MapTest {
-
- public static final class MySource extends SourceFunction> {
-
- @Override
- public void invoke(Collector> collector) throws Exception {
- for (int i = 0; i < 10; i++) {
- collector.collect(new Tuple1(i));
- }
- }
- }
-
- public static final class MySource1 extends SourceFunction> {
-
- @Override
- public void invoke(Collector> collector) throws Exception {
- for (int i = 0; i < 5; i++) {
- collector.collect(new Tuple1(i));
- }
- }
- }
-
- public static final class MySource2 extends SourceFunction> {
-
- @Override
- public void invoke(Collector> collector) throws Exception {
- for (int i = 5; i < 10; i++) {
- collector.collect(new Tuple1(i));
- }
- }
- }
-
- public static final class MySource3 extends SourceFunction> {
-
- @Override
- public void invoke(Collector> collector) throws Exception {
- for (int i = 10; i < 15; i++) {
- collector.collect(new Tuple1(i));
- }
- }
- }
-
- public static final class MyMap extends MapFunction, Tuple1> {
-
- @Override
- public Tuple1 map(Tuple1 value) throws Exception {
- map++;
- return new Tuple1(value.f0 * value.f0);
- }
- }
-
- public static final class MyJoinMap extends MapFunction, Tuple1> {
-
- @Override
- public Tuple1 map(Tuple1 value) throws Exception {
- joinSetResult.add(value.f0);
- return new Tuple1(value.f0);
- }
- }
-
- public static final class MyFieldsMap extends MapFunction, Tuple1> {
-
- private int counter = 0;
-
- @Override
- public Tuple1 map(Tuple1 value) throws Exception {
- counter++;
- if (counter == MAXSOURCE)
- allInOne = true;
- return new Tuple1(value.f0 * value.f0);
- }
- }
-
- public static final class MyDiffFieldsMap extends MapFunction, Tuple1> {
-
- private int counter = 0;
-
- @Override
- public Tuple1 map(Tuple1 value) throws Exception {
- counter++;
- if (counter > 3)
- threeInAll = false;
- return new Tuple1(value.f0 * value.f0);
- }
- }
-
- public static final class MySink extends SinkFunction> {
-
- @Override
- public void invoke(Tuple1 tuple) {
- result.add(tuple.f0);
- }
- }
-
- public static final class MyBroadcastSink extends SinkFunction> {
-
- @Override
- public void invoke(Tuple1 tuple) {
- broadcastResult++;
- }
- }
-
- public static final class MyShufflesSink extends SinkFunction> {
-
- @Override
- public void invoke(Tuple1 tuple) {
- shuffleResult++;
- }
- }
-
- public static final class MyFieldsSink extends SinkFunction> {
-
- @Override
- public void invoke(Tuple1 tuple) {
- fieldsResult++;
- }
- }
-
- public static final class MyDiffFieldsSink extends SinkFunction> {
-
- @Override
- public void invoke(Tuple1 tuple) {
- diffFieldsResult++;
- }
- }
-
- public static final class MyGraphSink extends SinkFunction> {
-
- @Override
- public void invoke(Tuple1 tuple) {
- graphResult++;
- }
- }
-
- public static final class JoinSink extends SinkFunction> {
-
- @Override
- public void invoke(Tuple1 tuple) {
- }
- }
-
- private static Set expected = new HashSet();
- private static Set result = new HashSet();
- private static int broadcastResult = 0;
- private static int shuffleResult = 0;
- private static int fieldsResult = 0;
- private static int diffFieldsResult = 0;
- private static int graphResult = 0;
- private static int map = 0;
- private static final int PARALELISM = 1;
- private static final int MAXSOURCE = 10;
- private static boolean allInOne = false;
- private static boolean threeInAll = true;
- private static Set fromCollectionSet = new HashSet();
- private static List fromCollectionFields = new ArrayList();
- private static Set fromCollectionDiffFieldsSet = new HashSet();
- private static Set singleJoinSetExpected = new HashSet();
- private static Set multipleJoinSetExpected = new HashSet();
- private static Set joinSetResult = new HashSet();
-
- private static void fillExpectedList() {
- for (int i = 0; i < 10; i++) {
- expected.add(i * i);
- }
- }
-
- private static void fillFromCollectionSet() {
- if(fromCollectionSet.isEmpty()){
- for (int i = 0; i < 10; i++) {
- fromCollectionSet.add(i);
- }
- }
- }
-
- private static void fillFromCollectionFieldsSet() {
- if(fromCollectionFields.isEmpty()){
- for (int i = 0; i < MAXSOURCE; i++) {
-
- fromCollectionFields.add(5);
- }
- }
- }
-
- private static void fillFromCollectionDiffFieldsSet() {
- if(fromCollectionDiffFieldsSet.isEmpty()){
- for (int i = 0; i < 9; i++) {
- fromCollectionDiffFieldsSet.add(i);
- }
- }
- }
-
- private static void fillSingleJoinSet() {
- for (int i = 0; i < 10; i++) {
- singleJoinSetExpected.add(i);
- }
- }
-
- private static void fillMultipleJoinSet() {
- for (int i = 0; i < 15; i++) {
- multipleJoinSetExpected.add(i);
- }
- }
-
-
- @Test
- public void mapTest() throws Exception {
-
- StreamExecutionEnvironment env = new StreamExecutionEnvironment();
-
- fillFromCollectionSet();
-
- DataStream> dataStream = env.fromCollection(fromCollectionSet)
- .map(new MyMap(), PARALELISM).addSink(new MySink());
-
- env.execute();
-
- fillExpectedList();
-
- assertTrue(expected.equals(result));
- }
-
- @Test
- public void broadcastSinkTest() throws Exception {
- StreamExecutionEnvironment env = new StreamExecutionEnvironment();
-
- fillFromCollectionSet();
-
- DataStream> dataStream = env
- .fromCollection(fromCollectionSet)
- .broadcast()
- .map(new MyMap(), 3)
- .addSink(new MyBroadcastSink());
-
- env.execute();
- assertEquals(30, broadcastResult);
-
- }
-
- @Test
- public void shuffleSinkTest() throws Exception {
- StreamExecutionEnvironment env = new StreamExecutionEnvironment();
-
- fillFromCollectionSet();
-
- DataStream> dataStream = env
- .fromCollection(fromCollectionSet)
- .map(new MyMap(), 3)
- .addSink(new MyShufflesSink());
- env.execute();
- assertEquals(10, shuffleResult);
-
- }
-
-// @Test
-// public void fieldsSinkTest() throws Exception {
-// StreamExecutionEnvironment env = new StreamExecutionEnvironment();
-// DataStream> dataStream = env
-// .addSource(new MySource(), 1)
-// .partitionBy(0)
-// .map(new MyMap(), 3)
-// .addSink(new MyFieldsSink());
-//
-// env.execute();
-// assertEquals(10, fieldsResult);
-//
-// }
-
- @Test
- public void fieldsMapTest() throws Exception {
- StreamExecutionEnvironment env = new StreamExecutionEnvironment();
-
- fillFromCollectionFieldsSet();
-
- DataStream> dataStream = env
- .fromCollection(fromCollectionFields)
- .partitionBy(0)
- .map(new MyFieldsMap(), 3)
- .addSink(new MyFieldsSink());
-
- env.execute();
- assertTrue(allInOne);
-
- }
-
- @Test
- public void diffFieldsMapTest() throws Exception {
- StreamExecutionEnvironment env = new StreamExecutionEnvironment();
-
- fillFromCollectionDiffFieldsSet();
-
- DataStream> dataStream = env
- .fromCollection(fromCollectionDiffFieldsSet)
- .partitionBy(0)
- .map(new MyDiffFieldsMap(), 3)
- .addSink(new MyDiffFieldsSink());
-
- env.execute();
- assertTrue(threeInAll);
- assertEquals(9, diffFieldsResult);
-
- }
-
-// @Test
-// public void graphTest() throws Exception {
-// for(int i=0; i<1000; i++){
-// System.out.println(i);
-// StreamExecutionEnvironment env = new StreamExecutionEnvironment();
-// DataStream> dataStream = env
-// .addSource(new MySource(), 2)
-// .partitionBy(0)
-// .map(new MyMap(), 3)
-// .broadcast()
-// .addSink(new MyGraphSink(),2);
-//
-// env.execute();
-// assertEquals(40, graphResult);
-// graphResult=0;
-// map=0;
-// }
-//
-// }
-
- @Test
- public void singleConnectWithTest() throws Exception {
-
- StreamExecutionEnvironment env = new StreamExecutionEnvironment();
-
- DataStream> source1 = env.addSource(new MySource1(),
- 1);
-
- DataStream> source2 = env
- .addSource(new MySource2(), 1)
- .connectWith(source1)
- .partitionBy(0)
- .map(new MyJoinMap(), 1)
- .addSink(new JoinSink());
-
- env.execute();
-
- fillSingleJoinSet();
-
- assertEquals(singleJoinSetExpected, joinSetResult);
- }
-
- @Test
- public void multipleConnectWithTest() throws Exception {
-
- StreamExecutionEnvironment env = new StreamExecutionEnvironment();
-
- DataStream> source1 = env.addSource(new MySource1(),
- 1);
-
- DataStream> source2 = env.addSource(new MySource2(),
- 1);
- DataStream> source3 = env
- .addSource(new MySource3(), 1)
- .connectWith(source1, source2)
- .partitionBy(0)
- .map(new MyJoinMap(), 1)
- .addSink(new JoinSink());
-
- env.execute();
-
- fillMultipleJoinSet();
-
- assertEquals(multipleJoinSetExpected, joinSetResult);
- }
-
-}
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/StreamCollectorManagerTest.java b/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/StreamCollectorManagerTest.java
deleted file mode 100644
index c8ca0e84aca44a7bb9286ab04632b5a462933b50..0000000000000000000000000000000000000000
--- a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/StreamCollectorManagerTest.java
+++ /dev/null
@@ -1,87 +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.streaming.api;
-
-import static org.junit.Assert.*;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.Test;
-
-import eu.stratosphere.api.java.tuple.Tuple;
-import eu.stratosphere.api.java.tuple.Tuple1;
-import eu.stratosphere.nephele.io.RecordWriter;
-import eu.stratosphere.streaming.api.streamcomponent.MockRecordWriter;
-import eu.stratosphere.streaming.api.streamrecord.StreamRecord;
-import eu.stratosphere.streaming.util.MockRecordWriterFactory;
-
-public class StreamCollectorManagerTest {
-
- StreamCollectorManager collector;
-
- @Test
- public void testCollect() {
- List batchSizesOfNotPartitioned = new ArrayList();
- List batchSizesOfPartitioned = new ArrayList();
- batchSizesOfPartitioned.add(2);
- batchSizesOfPartitioned.add(3);
- List parallelismOfOutput = new ArrayList();
- parallelismOfOutput.add(2);
- parallelismOfOutput.add(2);
- int keyPosition = 0;
- long batchTimeout = 1000;
- int channelID = 1;
-
- List> fOut = new ArrayList>();
-
- MockRecordWriter rw1 = MockRecordWriterFactory.create();
- MockRecordWriter rw2 = MockRecordWriterFactory.create();
-
- fOut.add(rw1);
- fOut.add(rw2);
-
- collector = new StreamCollectorManager(batchSizesOfNotPartitioned, batchSizesOfPartitioned, parallelismOfOutput, keyPosition, batchTimeout, channelID, null, fOut,fOut);
- Tuple1 t = new Tuple1();
-
- t.f0 = 0;
- collector.collect(t);
- t.f0 = 1;
- collector.collect(t);
- t.f0 = 0;
- collector.collect(t);
-
- StreamRecord r1 = rw1.emittedRecords.get(0);
- assertEquals(1, rw1.emittedRecords.size());
- assertEquals(0, r1.getTuple(0).getField(0));
- assertEquals(0, r1.getTuple(1).getField(0));
-
- t.f0 = 1;
- collector.collect(t);
-
- StreamRecord r2 = rw1.emittedRecords.get(1);
- assertEquals(2, rw1.emittedRecords.size());
- assertEquals(1, r2.getTuple(0).getField(0));
- assertEquals(1, r2.getTuple(1).getField(0));
-
- assertEquals(0, rw2.emittedRecords.size());
-
- t.f0 = 5;
- collector.collect(t);
- assertEquals(2, rw1.emittedRecords.size());
- assertEquals(1, rw2.emittedRecords.size());
- }
-}
diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/StreamCollectorTest.java b/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/StreamCollectorTest.java
deleted file mode 100755
index 812b21abb9ef3daa6c1d0eda27deab183b0497b5..0000000000000000000000000000000000000000
--- a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/StreamCollectorTest.java
+++ /dev/null
@@ -1,74 +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.streaming.api;
-
-import static org.junit.Assert.assertEquals;
-
-import org.junit.Test;
-
-import eu.stratosphere.api.java.tuple.Tuple1;
-import eu.stratosphere.streaming.api.streamcomponent.MockRecordWriter;
-import eu.stratosphere.streaming.util.MockRecordWriterFactory;
-
-public class StreamCollectorTest {
-
- @Test
- public void testStreamCollector() {
- StreamCollector> collector = new StreamCollector>(10, 1000, 0, null);
- assertEquals(10, collector.batchSize);
- }
-
- @Test
- public void testCollect() {
- StreamCollector> collector = new StreamCollector>(2, 1000, 0, null);
- collector.collect(new Tuple1(3));
- collector.collect(new Tuple1(4));
- collector.collect(new Tuple1(5));
- collector.collect(new Tuple1(6));
-
- }
-
- @Test
- public void testBatchSize() throws InterruptedException {
- StreamCollector> collector = new StreamCollector>(3, 100, 0, null);
- collector.collect(new Tuple1(0));
- collector.collect(new Tuple1(0));
- collector.collect(new Tuple1(0));
-
- Thread.sleep(200);
- collector.collect(new Tuple1(2));
- collector.collect(new Tuple1(3));
- }
-
- @Test
- public void recordWriter() {
- MockRecordWriter recWriter = MockRecordWriterFactory.create();
-
- StreamCollector> collector = new StreamCollector>(2, 1000, 0, null, recWriter);
- collector.collect(new Tuple1