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 clazz) { - setDeserializer(function, clazz); - setSerializer(function, clazz, 1); + } } - @SuppressWarnings({ "unchecked", "rawtypes" }) - private void setDeserializer(Object function, Class clazz) { + private void setSerializer(Object function, Class 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 userFunctionClass, - Configuration config) { - StreamComponent userFunction = null; - - byte[] userFunctionSerialized = config.getBytes("serializedudf", null); + public UserSinkInvokable getSinkInvokable(Configuration taskConfiguration) { + + Class 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 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 userFunctionClass = config.getClass("userfunction", - DefaultTaskInvokable.class, UserTaskInvokable.class); - return (UserTaskInvokable) getInvokable(userFunctionClass, config); + Class 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 userFunctionClass = config.getClass("userfunction", - DefaultSourceInvokable.class, UserSourceInvokable.class); - return (UserSourceInvokable) getInvokable(userFunctionClass, config); + Class 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> partitioner = config.getClass( - "partitionerClass_" + numberOfOutputs, DefaultPartitioner.class, - ChannelSelector.class); + Class> 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 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 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 recordType, + public StreamRecordReader(AbstractInvokable taskBase, Class 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 trueit at least one more record can be read from the - * associated input gate, otherwise false + * @return trueit 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(3)); - collector.collect(new Tuple1(4)); - collector.collect(new Tuple1(5)); - collector.collect(new Tuple1(6)); - - assertEquals((Integer) 3, recWriter.emittedRecords.get(0).getTuple(0).getField(0)); - assertEquals((Integer) 6, recWriter.emittedRecords.get(1).getTuple(1).getField(0)); - } - - @Test - public void testClose() { - } - -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/streamcomponent/MockRecordWriter.java b/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/streamcomponent/MockRecordWriter.java deleted file mode 100644 index e25d7d91b8a781341ca4e14d1142fb3e92b3f953..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-addons/src/test/java/eu/stratosphere/streaming/api/streamcomponent/MockRecordWriter.java +++ /dev/null @@ -1,40 +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.streamcomponent; - -import java.util.ArrayList; - -import eu.stratosphere.nephele.io.RecordWriter; -import eu.stratosphere.nephele.template.AbstractInputTask; -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 boolean initList() { - emittedRecords = new ArrayList(); - return true; - } - - @Override - public void emit(StreamRecord record) { - emittedRecords.add(record.copy()); - } -} \ No newline at end of file diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/pom.xml b/flink-addons/flink-streaming/stratosphere-streaming-core/pom.xml deleted file mode 100644 index 25d69b057f6bb6e1edbedfb91c483e1ce77b1bf2..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/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-core/src/test/java/eu/stratosphere/streaming/api/BatchReduceTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/BatchReduceTest.java deleted file mode 100644 index cd20a325f8de2e10700712f33815c6b0b1cb1855..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/BatchReduceTest.java +++ /dev/null @@ -1,86 +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 java.util.ArrayList; -import java.util.Iterator; - -import org.junit.Test; - -import eu.stratosphere.api.java.functions.GroupReduceFunction; -import eu.stratosphere.api.java.tuple.Tuple1; -import eu.stratosphere.util.Collector; - -public class BatchReduceTest { - - private static ArrayList avgs = new ArrayList(); - private static final int BATCH_SIZE = 5; - private static final int PARALELISM = 1; - - public static final class MyBatchReduce extends - GroupReduceFunction, Tuple1> { - - @Override - public void reduce(Iterator> values, Collector> out) - throws Exception { - - Double sum = 0.; - Double count = 0.; - while (values.hasNext()) { - sum += values.next().f0; - count++; - } - - out.collect(new Tuple1(sum / count)); - } - } - - public static final class MySink extends SinkFunction> { - private static final long serialVersionUID = 1L; - - @Override - public void invoke(Tuple1 tuple) { - avgs.add(tuple.f0); - } - - } - - public static final class MySource extends SourceFunction> { - private static final long serialVersionUID = 1L; - - @Override - public void invoke(Collector> collector) { - for (Double i = 1.; i <= 100; i++) { - collector.collect(new Tuple1(i)); - } - } - } - - @Test - public void test() throws Exception { - StreamExecutionEnvironment env = new StreamExecutionEnvironment(); - DataStream> dataStream0 = env.addSource(new MySource(),1) - .batchReduce(new MyBatchReduce(), BATCH_SIZE, PARALELISM).addSink(new MySink()); - - env.execute(); - - for (int i = 0; i < avgs.size(); i++) { - assertEquals(3.0 + i * BATCH_SIZE, avgs.get(i), 0); - } - } -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/BatchTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/BatchTest.java deleted file mode 100644 index f83af5a28313d861a54019344bb39ff2f83dd0d6..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/BatchTest.java +++ /dev/null @@ -1,106 +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 org.junit.Test; - -import eu.stratosphere.api.java.functions.FlatMapFunction; -import eu.stratosphere.api.java.tuple.Tuple1; -import eu.stratosphere.util.Collector; - -public class BatchTest { - - private static final int PARALLELISM = 1; - private static final int SOURCE_PARALELISM = 1; - private static final int SINK_PARALELISM = 5; - private static int count = 0; - private static boolean partitionCorrect = true; - - private static final class MySource extends SourceFunction> { - - private Tuple1 outTuple = new Tuple1(); - - @Override - public void invoke(Collector> collector) throws Exception { - for (int i = 0; i < 20; i++) { - outTuple.f0 = "string #" + i; - collector.collect(outTuple); - } - } - } - - private static final class MyMap extends FlatMapFunction, Tuple1> { - - @Override - public void flatMap(Tuple1 value, Collector> out) throws Exception { - out.collect(value); - } - } - - private static final class MySink extends SinkFunction> { - - @Override - public void invoke(Tuple1 tuple) { - count++; - } - } - - private static final class MyPartitionSink extends SinkFunction> { - - int hash=-1000; - @Override - public void invoke(Tuple1 tuple) { - if(hash==-1000) hash=tuple.f0.hashCode() % SINK_PARALELISM; - else{ - if(hash!=tuple.f0.hashCode() % SINK_PARALELISM) partitionCorrect=false; - } - } - } - - @Test - public void test() throws Exception { - StreamExecutionEnvironment env = new StreamExecutionEnvironment(); - - DataStream> dataStream = env - .addSource(new MySource(), SOURCE_PARALELISM) - .flatMap(new MyMap(), PARALLELISM).batch(4) - .flatMap(new MyMap(), PARALLELISM).batch(2) - .flatMap(new MyMap(), PARALLELISM).batch(5) - .flatMap(new MyMap(), PARALLELISM).batch(4) - .addSink(new MySink()); - - env.execute(); - - assertEquals(20, count); - } - - @Test - public void partitionTest() throws Exception { - StreamExecutionEnvironment env = new StreamExecutionEnvironment(); - - DataStream> dataStream = env - .addSource(new MySource(), SOURCE_PARALELISM) - .flatMap(new MyMap(), PARALLELISM).batch(4) - .partitionBy(0) - .addSink(new MyPartitionSink(), SINK_PARALELISM); - - env.execute(); - - assertTrue(partitionCorrect); - } -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/FlatMapTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/FlatMapTest.java deleted file mode 100644 index 303c96249e58651242e024c5a8a28e48735f5184..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/FlatMapTest.java +++ /dev/null @@ -1,242 +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.io.ByteArrayInputStream; -import java.io.ObjectInputStream; -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.FlatMapFunction; -import eu.stratosphere.api.java.tuple.Tuple; -import eu.stratosphere.api.java.tuple.Tuple1; -import eu.stratosphere.api.java.tuple.Tuple2; -import eu.stratosphere.api.java.typeutils.TupleTypeInfo; -import eu.stratosphere.api.java.typeutils.TypeExtractor; -import eu.stratosphere.configuration.Configuration; -import eu.stratosphere.nephele.jobgraph.AbstractJobVertex; -import eu.stratosphere.nephele.jobgraph.JobInputVertex; -import eu.stratosphere.nephele.jobgraph.JobOutputVertex; -import eu.stratosphere.nephele.jobgraph.JobTaskVertex; -import eu.stratosphere.streaming.api.MapTest.MyMap; -import eu.stratosphere.streaming.api.MapTest.MySink; -import eu.stratosphere.streaming.api.PrintTest.MyFlatMap; -import eu.stratosphere.streaming.api.invokable.UserSinkInvokable; -import eu.stratosphere.streaming.api.invokable.UserSourceInvokable; -import eu.stratosphere.streaming.api.invokable.UserTaskInvokable; -import eu.stratosphere.util.Collector; - -public class FlatMapTest { - - public static final class MyFlatMap extends FlatMapFunction, Tuple1> { - - @Override - public void flatMap(Tuple1 value, Collector> out) throws Exception { - out.collect(new Tuple1(value.f0 * value.f0)); - - } - - } - - public static final class ParallelFlatMap extends - FlatMapFunction, Tuple1> { - - @Override - public void flatMap(Tuple1 value, Collector> out) throws Exception { - numberOfElements++; - - } - - } - - public static final class GenerateSequenceFlatMap extends - FlatMapFunction, Tuple1> { - - @Override - public void flatMap(Tuple1 value, Collector> out) throws Exception { - out.collect(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 FromElementsSink extends SinkFunction> { - - @Override - public void invoke(Tuple1 tuple) { - fromElementsResult.add(tuple.f0); - } - - } - - public static final class FromCollectionSink extends SinkFunction> { - - @Override - public void invoke(Tuple1 tuple) { - fromCollectionResult.add(tuple.f0); - } - - } - - public static final class GenerateSequenceSink extends SinkFunction> { - - @Override - public void invoke(Tuple1 tuple) { - generateSequenceResult.add(tuple.f0); - } - - } - - private static void fillExpectedList() { - for (int i = 0; i < 10; i++) { - expected.add(i * i); - } - } - - private static void fillFromElementsExpected() { - fromElementsExpected.add(4); - fromElementsExpected.add(25); - fromElementsExpected.add(81); - } - - private static void fillSequenceSet() { - for (int i = 0; i < 10; i++) { - sequenceExpected.add(i * i); - } - } - - private static void fillLongSequenceSet() { - for (int i = 0; i < 10; i++) { - sequenceLongExpected.add((long)(i * i)); - } - } - - private static void fillFromCollectionSet() { - if(fromCollectionSet.isEmpty()){ - for (int i = 0; i < 10; i++) { - fromCollectionSet.add(i); - } - } - } - - private static final int PARALELISM = 1; - private static int numberOfElements = 0; - private static Set expected = new HashSet(); - private static Set result = new HashSet(); - private static Set fromElementsExpected = new HashSet(); - private static Set fromElementsResult = new HashSet(); - private static Set fromCollectionSet = new HashSet(); - private static Set sequenceExpected = new HashSet(); - private static Set sequenceLongExpected = new HashSet(); - private static Set fromCollectionResult = new HashSet(); - private static Set generateSequenceResult = new HashSet(); - - @Test - public void test() throws Exception { - StreamExecutionEnvironment env = new StreamExecutionEnvironment(2, 1000); - - fillFromCollectionSet(); - - DataStream> dataStream = env.fromCollection(fromCollectionSet) - .flatMap(new MyFlatMap(), PARALELISM).addSink(new MySink()); - - env.execute(); - - fillExpectedList(); - - assertTrue(expected.equals(result)); - - } - - @Test - public void parallelShuffleconnectTest() throws Exception { - StreamExecutionEnvironment env = new StreamExecutionEnvironment(); - - fillFromCollectionSet(); - - DataStream> source = env.fromCollection(fromCollectionSet); - DataStream> map = source.flatMap(new ParallelFlatMap(), 1).addSink( - new MySink()); - DataStream> map2 = source.flatMap(new ParallelFlatMap(), 1).addSink( - new MySink()); - - env.execute(); - - assertEquals(20, numberOfElements); - numberOfElements=0; - - - } - - @Test - public void fromElementsTest() throws Exception { - StreamExecutionEnvironment env = new StreamExecutionEnvironment(); - DataStream> map = env.fromElements(2, 5, 9).flatMap(new MyFlatMap(), 1); - DataStream> sink = map.addSink(new FromElementsSink()); - - fillFromElementsExpected(); - - env.execute(); - assertEquals(fromElementsExpected, fromElementsResult); - - } - - @Test - public void fromCollectionTest() throws Exception { - StreamExecutionEnvironment env = new StreamExecutionEnvironment(); - - fillFromCollectionSet(); - - DataStream> map = env.fromCollection(fromCollectionSet).flatMap( - new MyFlatMap(), 1); - DataStream> sink = map.addSink(new FromCollectionSink()); - - fillSequenceSet(); - - env.execute(); - assertEquals(sequenceExpected, fromCollectionResult); - - } - - @Test - public void generateSequenceTest() throws Exception { - StreamExecutionEnvironment env = new StreamExecutionEnvironment(); - - DataStream> map = env.generateSequence(0, 9).flatMap( - new GenerateSequenceFlatMap(), 1); - DataStream> sink = map.addSink(new GenerateSequenceSink()); - - fillLongSequenceSet(); - - env.execute(); - assertEquals(sequenceLongExpected, generateSequenceResult); - } -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/PrintTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/PrintTest.java deleted file mode 100755 index d636573ed0b213feefea3bbefa45d876bc967dfa..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/PrintTest.java +++ /dev/null @@ -1,58 +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 java.util.HashSet; -import java.util.Set; - -import org.junit.Test; - -import eu.stratosphere.api.java.functions.FlatMapFunction; -import eu.stratosphere.api.java.tuple.Tuple2; -import eu.stratosphere.util.Collector; - -public class PrintTest { - - public static final class MyFlatMap extends - FlatMapFunction, Tuple2> { - - private static final long serialVersionUID = 1L; - - @Override - public void flatMap(Tuple2 value, Collector> out) - throws Exception { - out.collect(new Tuple2(value.f0 * value.f0, value.f1)); - - } - - } - - @Test - public void test() throws Exception { - - StreamExecutionEnvironment env = new StreamExecutionEnvironment(); - env.fromElements(2, 3, 4).print(); - env.generateSequence(1, 10).print(); - Set a = new HashSet(); - a.add(-2); - a.add(-100); - env.fromCollection(a).print(); - env.execute(); - - } - - -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/TypeExtractTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/TypeExtractTest.java deleted file mode 100755 index f892ea2a55d7ae25721c566ed7d85aaa8f3320dc..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/TypeExtractTest.java +++ /dev/null @@ -1,67 +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 java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.io.Serializable; - -import org.junit.Test; - -import eu.stratosphere.api.java.tuple.Tuple; -import eu.stratosphere.api.java.typeutils.TupleTypeInfo; -import eu.stratosphere.api.java.typeutils.TypeExtractor; -import eu.stratosphere.types.TypeInformation; - -public class TypeExtractTest { - - public static class MySuperlass implements Serializable{ - - } - - public static class Myclass extends MySuperlass { - - private static final long serialVersionUID = 1L; - - } - - @Test - public void test() throws IOException, ClassNotFoundException { - - Myclass f = new Myclass(); - - System.out.println(f.getClass().getGenericSuperclass()); - TypeInformation ts = TypeExtractor.createTypeInfo(MySuperlass.class, f.getClass(), 0, - null, null); - - System.out.println(ts); - - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ObjectOutputStream oos; - - oos = new ObjectOutputStream(baos); - oos.writeObject(f); - - ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray())); - - - System.out.println(new TupleTypeInfo(TypeExtractor.getForObject(in.readObject()))); - } - -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentTest.java deleted file mode 100644 index 73f2223be821ef698741fa6943f50769cd9041e1..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamcomponent/StreamComponentTest.java +++ /dev/null @@ -1,110 +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.streamcomponent; - -import static org.junit.Assert.assertEquals; - -import java.util.HashMap; -import java.util.Map; - -import org.junit.BeforeClass; -import org.junit.Test; - -import eu.stratosphere.api.java.functions.MapFunction; -import eu.stratosphere.api.java.tuple.Tuple1; -import eu.stratosphere.api.java.tuple.Tuple2; -import eu.stratosphere.streaming.api.DataStream; -import eu.stratosphere.streaming.api.SinkFunction; -import eu.stratosphere.streaming.api.SourceFunction; -import eu.stratosphere.streaming.api.StreamExecutionEnvironment; -import eu.stratosphere.util.Collector; - -public class StreamComponentTest { - - private static final int PARALELISM = 1; - private static final int SOURCE_PARALELISM = 1; - - public static Map data = new HashMap(); - - public static class MySource extends SourceFunction> { - private static final long serialVersionUID = 1L; - - private Tuple1 tuple = new Tuple1(0); - - @Override - public void invoke(Collector> collector) throws Exception { - for (int i = 0; i < 10; i++) { - tuple.f0 = i; - collector.collect(tuple); - System.out.println("collecting " + tuple); - } - } - } - - public static class MyTask extends MapFunction, Tuple2> { - private static final long serialVersionUID = 1L; - - @Override - public Tuple2 map(Tuple1 value) throws Exception { - Integer i = value.f0; - System.out.println("mapping " + i); - return new Tuple2(i, i + 1); - } - } - - // TODO test multiple tasks - // public static class MyOtherTask extends MapFunction, - // Tuple2> { - // private static final long serialVersionUID = 1L; - // - // @Override - // public Tuple2 map(Tuple1 value) throws - // Exception { - // Integer i = value.f0; - // return new Tuple2(-i - 1, -i - 2); - // } - // } - - public static class MySink extends SinkFunction> { - private static final long serialVersionUID = 1L; - - @Override - public void invoke(Tuple2 tuple) { - Integer k = tuple.getField(0); - Integer v = tuple.getField(1); - data.put(k, v); - } - } - - @BeforeClass - public static void runStream() { - StreamExecutionEnvironment context = new StreamExecutionEnvironment(); - - DataStream> oneTask = context.addSource(new MySource(), SOURCE_PARALELISM) - .map(new MyTask(), PARALELISM).addSink(new MySink()); - - context.execute(); - } - - @Test - public void test() { - assertEquals(10, data.keySet().size()); - - for (Integer k : data.keySet()) { - assertEquals((Integer) (k + 1), data.get(k)); - } - } -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamrecord/ArrayStreamRecordTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamrecord/ArrayStreamRecordTest.java deleted file mode 100755 index e81fce48e680b1db1e5454019247f1cf05349fab..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamrecord/ArrayStreamRecordTest.java +++ /dev/null @@ -1,124 +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.streamrecord; - -import static org.junit.Assert.assertEquals; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; - -import org.junit.Test; - -import eu.stratosphere.api.java.tuple.Tuple; -import eu.stratosphere.api.java.tuple.Tuple1; -import eu.stratosphere.api.java.tuple.Tuple2; -import eu.stratosphere.api.java.typeutils.TypeExtractor; -import eu.stratosphere.types.TypeInformation; - -public class ArrayStreamRecordTest { - - @Test - public void constructorTest() { - ArrayStreamRecord record = new ArrayStreamRecord(10); - assertEquals(10, record.getBatchSize()); - - Tuple[] tuples = new Tuple[2]; - tuples[0] = new Tuple1(2); - tuples[1] = new Tuple1(3); - - ArrayStreamRecord record1 = new ArrayStreamRecord(tuples); - - assertEquals(2, record1.getBatchSize()); - - ArrayStreamRecord record2 = new ArrayStreamRecord(record1); - assertEquals(2, record2.getBatchSize()); - } - - @Test - public void typeExtractTest() throws IOException, ClassNotFoundException { - - ByteArrayOutputStream buff = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(buff); - - MyGeneric g = new MyGeneric2(); - out.writeObject(g); - - ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(buff.toByteArray())); - - MyGeneric f = (MyGeneric) in.readObject(); - - TypeInformation ti = TypeExtractor.createTypeInfo(MyGeneric.class, f.getClass(), 0, - null, null); - - System.out.println("Type info: " + ti); - - } - - @Test - public void StreamRecordSpeedTest() { - int len = 100000; - ArrayStreamRecord arecord = new ArrayStreamRecord(len); - StreamRecord record = new ListStreamRecord(len); - Tuple2 tuple = new Tuple2(2, "a"); - long standardTime=System.nanoTime(); - - for (int i = 0; i < len; i++) { - record.setTuple(i, tuple); - } - standardTime=System.nanoTime()-standardTime; - - long arrayTime=System.nanoTime(); - for (int i = 0; i < len; i++) { - arecord.setTuple(i, tuple); - } - arrayTime=System.nanoTime()-arrayTime; - - System.out.println("Standard time: "+standardTime); - System.out.println("Array time: "+arrayTime); - - float multi = (float)standardTime/(float)arrayTime; - System.out.println("Mulitplier: "+multi); - - } - - @Test - public void truncatedSizeTest() { - StreamRecord record = new ArrayStreamRecord(4); - record.setTuple(0, new Tuple1(0)); - record.setTuple(1, new Tuple1(1)); - record.setTuple(2, new Tuple1(2)); - record.setTuple(3, new Tuple1(3)); - - StreamRecord truncatedRecord = new ArrayStreamRecord(record, 2); - assertEquals(2, truncatedRecord.batchSize); - assertEquals(0, truncatedRecord.getTuple(0).getField(0)); - assertEquals(1, truncatedRecord.getTuple(1).getField(0)); - } - - @Test - public void copyTupleTest() { - Tuple1 t1 = new Tuple1("T1"); - Tuple1 t2 = (Tuple1) StreamRecord.copyTuple(t1); - assertEquals("T1", t2.f0); - - t2.f0 = "T2"; - assertEquals("T1", t1.f0); - assertEquals("T2", t2.f0); - } -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamrecord/ListStreamRecordTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamrecord/ListStreamRecordTest.java deleted file mode 100644 index 52a921fa1c76ccf196b9661c3ba93d9ce6624fef..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamrecord/ListStreamRecordTest.java +++ /dev/null @@ -1,76 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 20102014 by the Stratosphere project (http://stratosphere.eu) - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.streamrecord; - -import static org.junit.Assert.assertEquals; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -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.api.java.typeutils.TupleTypeInfo; -import eu.stratosphere.api.java.typeutils.TypeExtractor; -import eu.stratosphere.types.TypeInformation; - -public class ListStreamRecordTest { - - @Test - public void constructorTest() { - StreamRecord record = new ListStreamRecord(10); - assertEquals(10, record.getBatchSize()); - - List tuples = new ArrayList(); - tuples.add(new Tuple1(2)); - tuples.add(new Tuple1(3)); - - StreamRecord record1 = new ListStreamRecord(tuples); - - assertEquals(2, record1.getBatchSize()); - - StreamRecord record2 = new ListStreamRecord(record1); - assertEquals(2, record2.getBatchSize()); - } - - @Test - public void typeExtractTest() throws IOException, ClassNotFoundException { - - ByteArrayOutputStream buff = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(buff); - - MyGeneric g = new MyGeneric2(); - out.writeObject(g); - - ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(buff.toByteArray())); - - MyGeneric f = (MyGeneric) in.readObject(); - - TypeInformation ti = TypeExtractor.createTypeInfo(MyGeneric.class, f.getClass(), 0, - null, null); - TupleTypeInfo tt= (TupleTypeInfo)ti; - - System.out.println("Type info: " + tt); - - } - -} \ No newline at end of file diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamrecord/MyGeneric.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamrecord/MyGeneric.java deleted file mode 100755 index 83a8e94b3a8338316416c871104078c07884caa6..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamrecord/MyGeneric.java +++ /dev/null @@ -1,24 +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.streamrecord; - -import java.io.Serializable; - -public abstract class MyGeneric implements Serializable { - - public abstract void asd(); - -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamrecord/MyGeneric2.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamrecord/MyGeneric2.java deleted file mode 100755 index 38c0d03c57b1068dc8d302c9963334c40bc86f45..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamrecord/MyGeneric2.java +++ /dev/null @@ -1,27 +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.streamrecord; - -import eu.stratosphere.api.java.tuple.Tuple1; - -public class MyGeneric2 extends MyGeneric> { - - @Override - public void asd() { - - } - -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamrecord/UUIDTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamrecord/UUIDTest.java deleted file mode 100644 index 482ac8d4b75ce02cd29f202daa91e7939479cae8..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/api/streamrecord/UUIDTest.java +++ /dev/null @@ -1,47 +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.streamrecord; - -import static org.junit.Assert.*; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; - -import org.junit.Test; - -public class UUIDTest { - - @Test - public void test() throws IOException { - ByteArrayOutputStream buff = new ByteArrayOutputStream(); - DataOutputStream out = new DataOutputStream(buff); - - UID id = new UID(3); - id.write(out); - - DataInputStream in = new DataInputStream(new ByteArrayInputStream(buff.toByteArray())); - - UID id2 = new UID(); - id2.read(in); - - assertEquals(id.getChannelId(), id2.getChannelId()); - assertArrayEquals(id.getGeneratedId(), id2.getGeneratedId()); - assertArrayEquals(id.getId(), id2.getId()); - } - -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/faulttolerance/AtLeastOnceBufferTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/faulttolerance/AtLeastOnceBufferTest.java deleted file mode 100755 index 6162977c781e0c23a0639b0a309c4549ebff9c52..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/faulttolerance/AtLeastOnceBufferTest.java +++ /dev/null @@ -1,240 +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.faulttolerance; - -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import org.junit.Before; -import org.junit.Test; - -import eu.stratosphere.api.java.tuple.Tuple1; -import eu.stratosphere.streaming.api.streamrecord.ArrayStreamRecord; -import eu.stratosphere.streaming.api.streamrecord.StreamRecord; -import eu.stratosphere.streaming.api.streamrecord.UID; - -public class AtLeastOnceBufferTest { - - AtLeastOnceFaultToleranceBuffer buffer; - int[] numberOfChannels; - - @Before - public void setUp() throws Exception { - - numberOfChannels = new int[] { 1, 2 }; - - buffer = new AtLeastOnceFaultToleranceBuffer(numberOfChannels, 1); - - } - - @Test - public void testAddToAckCounter() { - - StreamRecord record1 = new ArrayStreamRecord(1).setTuple(0, new Tuple1("R1")).setId(1); - UID id = record1.getId(); - - buffer.addToAckCounter(record1.getId()); - - assertEquals((Integer) 3, buffer.ackCounter.get(id)); - - } - - @Test - public void testRemoveFromAckCounter() { - StreamRecord record1 = new ArrayStreamRecord(1).setTuple(0, new Tuple1("R1")).setId(1); - StreamRecord record2 = new ArrayStreamRecord(1).setTuple(0, new Tuple1("R2")).setId(1); - - UID id = record1.getId(); - - buffer.addToAckCounter(record1.getId()); - buffer.addToAckCounter(record2.getId()); - - record1.setId(1); - - buffer.removeFromAckCounter(record2.getId()); - - assertEquals((Integer) 3, buffer.ackCounter.get(id)); - assertFalse(buffer.ackCounter.containsKey(record2.getId())); - - } - - @Test - public void testAck() { - StreamRecord record1 = new ArrayStreamRecord(1).setTuple(0, new Tuple1("R1")).setId(1); - UID id = record1.getId(); - - buffer.add(record1); - assertEquals((Integer) 3, buffer.ackCounter.get(id)); - - buffer.ack(id, 1); - assertEquals((Integer) 2, buffer.ackCounter.get(id)); - - buffer.ack(id, 1); - assertEquals((Integer) 1, buffer.ackCounter.get(id)); - - buffer.ack(id, 1); - assertFalse(buffer.ackCounter.containsKey(id)); - assertFalse(buffer.recordBuffer.containsKey(id)); - assertFalse(buffer.recordTimestamps.containsKey(id)); - - } - - @Test - public void testAtLeastOnceFaultToleranceBuffer() { - numberOfChannels = new int[] { 2, 2, 2 }; - - buffer = new AtLeastOnceFaultToleranceBuffer(numberOfChannels, 2); - - assertArrayEquals(numberOfChannels, buffer.numberOfEffectiveChannels); - assertEquals(2, buffer.componentInstanceID); - assertEquals(6, buffer.totalNumberOfEffectiveChannels); - - } - - @Test - public void testAdd() { - - StreamRecord record1 = new ArrayStreamRecord(1).setId(1); - record1.setTuple(0, new Tuple1("R1")); - - UID id1 = record1.getId().copy(); - - Long nt = System.nanoTime(); - - buffer.add(record1); - System.out.println(id1); - System.out.println(buffer.ackCounter); - - System.out.println("ADD - " + " exec. time (ns): " + (System.nanoTime() - nt)); - - record1.setTuple(0, new Tuple1("R2")); - record1.setId(1); - UID id2 = record1.getId().copy(); - - try { - Thread.sleep(10); - } catch (InterruptedException e) { - } - buffer.add(record1); - System.out.println(id1); - System.out.println(buffer.ackCounter); - System.out.println(buffer.recordBuffer); - - assertEquals((Integer) 3, buffer.ackCounter.get(id1)); - assertEquals((Integer) 3, buffer.ackCounter.get(id2)); - - assertEquals("R1", buffer.recordBuffer.get(id1).getTuple(0).getField(0)); - assertEquals(id1, buffer.recordBuffer.get(id1).getId()); - - assertEquals("R2", buffer.recordBuffer.get(id2).getTuple(0).getField(0)); - assertEquals(id2, buffer.recordBuffer.get(id2).getId()); - - assertEquals(2, buffer.recordTimestamps.size()); - assertEquals(2, buffer.recordsByTime.size()); - assertEquals(2, buffer.recordBuffer.size()); - assertEquals(2, buffer.ackCounter.size()); - - } - - @Test - public void testFail() { - StreamRecord record1 = new ArrayStreamRecord(1).setTuple(0, new Tuple1("R1")).setId(1); - UID id1 = record1.getId(); - - buffer.add(record1); - buffer.ack(id1, 1); - buffer.ack(id1, 1); - - assertEquals(1, buffer.recordBuffer.size()); - assertEquals(1, buffer.recordTimestamps.size()); - assertEquals(1, buffer.ackCounter.size()); - - StreamRecord failed = buffer.fail(id1); - UID id2 = failed.getId(); - - assertFalse(buffer.ackCounter.containsKey(id1)); - assertFalse(buffer.recordBuffer.containsKey(id1)); - assertFalse(buffer.recordTimestamps.containsKey(id1)); - - assertTrue(buffer.ackCounter.containsKey(id2)); - assertTrue(buffer.recordBuffer.containsKey(id2)); - assertTrue(buffer.recordTimestamps.containsKey(id2)); - - assertEquals(1, buffer.recordBuffer.size()); - assertEquals(1, buffer.recordTimestamps.size()); - assertEquals(1, buffer.ackCounter.size()); - } - - //TODO fix test -// @Test -// public void testAddTimestamp() { -// -// Long ctime = System.currentTimeMillis(); -// -// UID id = new UID(1); -// buffer.addTimestamp(id); -// -// assertEquals(ctime, buffer.recordTimestamps.get(id)); -// -// assertTrue(buffer.recordsByTime.containsKey(ctime)); -// assertTrue(buffer.recordsByTime.get(ctime).contains(id)); -// } - - @Test - public void testRemove() { - StreamRecord record1 = new ArrayStreamRecord(1).setTuple(0, new Tuple1("R1")).setId(1); - - UID id1 = record1.getId(); - buffer.add(record1); - - record1.setTuple(0, new Tuple1("R2")); - record1.setId(1); - UID id2 = record1.getId(); - buffer.add(record1); - - assertTrue(buffer.ackCounter.containsKey(id1)); - assertTrue(buffer.recordBuffer.containsKey(id1)); - assertTrue(buffer.recordTimestamps.containsKey(id1)); - - assertTrue(buffer.ackCounter.containsKey(id2)); - assertTrue(buffer.recordBuffer.containsKey(id2)); - assertTrue(buffer.recordTimestamps.containsKey(id2)); - - assertEquals(2, buffer.recordBuffer.size()); - assertEquals(2, buffer.recordTimestamps.size()); - assertEquals(2, buffer.ackCounter.size()); - - StreamRecord removed = buffer.remove(id1); - assertEquals("R1", removed.getTuple(0).getField(0)); - assertEquals(id1, removed.getId()); - - assertFalse(buffer.ackCounter.containsKey(id1)); - assertFalse(buffer.recordBuffer.containsKey(id1)); - assertFalse(buffer.recordTimestamps.containsKey(id1)); - - assertTrue(buffer.ackCounter.containsKey(id2)); - assertTrue(buffer.recordBuffer.containsKey(id2)); - assertTrue(buffer.recordTimestamps.containsKey(id2)); - - assertEquals(1, buffer.recordBuffer.size()); - assertEquals(1, buffer.recordTimestamps.size()); - assertEquals(1, buffer.ackCounter.size()); - - } - -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/faulttolerance/ExactlyOnceBufferTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/faulttolerance/ExactlyOnceBufferTest.java deleted file mode 100755 index 832d25f573553dc23210454fdf2e4859691494f2..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/faulttolerance/ExactlyOnceBufferTest.java +++ /dev/null @@ -1,155 +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.faulttolerance; - -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import org.junit.Before; -import org.junit.Test; - -import eu.stratosphere.api.java.tuple.Tuple1; -import eu.stratosphere.streaming.api.streamrecord.ArrayStreamRecord; -import eu.stratosphere.streaming.api.streamrecord.StreamRecord; -import eu.stratosphere.streaming.api.streamrecord.UID; - -public class ExactlyOnceBufferTest { - - ExactlyOnceFaultToleranceBuffer buffer; - int[] numberOfChannels; - - @Before - public void setUp() throws Exception { - numberOfChannels = new int[] { 1, 2, 2 }; - buffer = new ExactlyOnceFaultToleranceBuffer(numberOfChannels, 1); - } - - @Test - public void testAddToAckCounter() { - StreamRecord record1 = new ArrayStreamRecord(1).setTuple(0, new Tuple1("R1")).setId(1); - buffer.addToAckCounter(record1.getId()); - assertArrayEquals(new int[] { 0, 1, 2, 2 }, buffer.ackCounter.get(record1.getId())); - - StreamRecord record2 = new ArrayStreamRecord(1).setTuple(0, new Tuple1("R2")).setId(1); - buffer.addToAckCounter(record2.getId(), 1); - int[] acks = buffer.ackCounter.get(record2.getId()); - assertArrayEquals(new int[] { 2, 0, 2, 0 }, acks); - - StreamRecord record3 = new ArrayStreamRecord(1).setTuple(0, new Tuple1("R3")).setId(1); - buffer.addToAckCounter(record3.getId(), 0); - acks = buffer.ackCounter.get(record3.getId()); - assertArrayEquals(new int[] { 2, 1, 0, 0 }, acks); - } - - @Test - public void testRemoveFromAckCounter() { - StreamRecord record1 = new ArrayStreamRecord(1).setTuple(0, new Tuple1("R1")).setId(1); - StreamRecord record2 = new ArrayStreamRecord(1).setTuple(0, new Tuple1("R2")).setId(1); - - buffer.addToAckCounter(record1.getId()); - buffer.addToAckCounter(record2.getId()); - assertEquals(2, buffer.ackCounter.size()); - - buffer.removeFromAckCounter(record2.getId()); - - assertEquals(1, buffer.ackCounter.size()); - assertArrayEquals(new int[] { 0, 1, 2, 2 }, buffer.ackCounter.get(record1.getId())); - assertFalse(buffer.ackCounter.containsKey(record2.getId())); - - StreamRecord record3 = new ArrayStreamRecord(1).setTuple(0, new Tuple1("R3")).setId(1); - StreamRecord record4 = new ArrayStreamRecord(1).setTuple(0, new Tuple1("R4")).setId(1); - - buffer.addToAckCounter(record3.getId(), 0); - buffer.addToAckCounter(record4.getId(), 2); - assertEquals(3, buffer.ackCounter.size()); - - buffer.removeFromAckCounter(record3.getId()); - assertEquals(2, buffer.ackCounter.size()); - assertTrue(buffer.ackCounter.containsKey(record4.getId())); - assertFalse(buffer.ackCounter.containsKey(record3.getId())); - } - - @Test - public void testAck() { - StreamRecord record1 = new ArrayStreamRecord(1).setTuple(0, new Tuple1("R1")).setId(1); - UID id = record1.getId(); - buffer.add(record1); - - assertArrayEquals(new int[] { 0, 1, 2, 2 }, buffer.ackCounter.get(id)); - buffer.ack(id, 0); - assertArrayEquals(new int[] { 1, 0, 2, 2 }, buffer.ackCounter.get(id)); - buffer.ack(id, 1); - assertArrayEquals(new int[] { 1, 0, 1, 2 }, buffer.ackCounter.get(id)); - buffer.ack(id, 1); - assertArrayEquals(new int[] { 2, 0, 0, 2 }, buffer.ackCounter.get(id)); - buffer.ack(id, 2); - buffer.ack(id, 2); - assertFalse(buffer.ackCounter.containsKey(id)); - } - - @Test - public void testFailChannel() { - StreamRecord record1 = new ArrayStreamRecord(1).setTuple(0, new Tuple1("R1")).setId(1); - UID id = record1.getId(); - buffer.add(record1); - - assertArrayEquals(new int[] { 0, 1, 2, 2 }, buffer.ackCounter.get(id)); - StreamRecord failedRecord = buffer.failChannel(id, 1); - - assertArrayEquals(new int[] { 1, 1, 0, 2 }, buffer.ackCounter.get(id)); - assertArrayEquals(new int[] { 2, 0, 2, 0 }, buffer.ackCounter.get(failedRecord.getId())); - assertEquals(2, buffer.ackCounter.size()); - buffer.ack(id, 1); - assertArrayEquals(new int[] { 1, 1, -1, 2 }, buffer.ackCounter.get(id)); - assertEquals(2, buffer.ackCounter.size()); - - assertEquals(null, buffer.failChannel(id, 1)); - assertArrayEquals(new int[] { 1, 1, -1, 2 }, buffer.ackCounter.get(id)); - assertArrayEquals(new int[] { 2, 0, 2, 0 }, buffer.ackCounter.get(failedRecord.getId())); - assertEquals(2, buffer.ackCounter.size()); - - buffer.failChannel(failedRecord.getId(), 1); - assertFalse(buffer.ackCounter.containsKey(failedRecord.getId())); - } - - @Test - public void testExactlyOnceFaultToleranceBuffer() { - // fail("Not yet implemented"); - } - - @Test - public void testFaultToleranceBuffer() { - // fail("Not yet implemented"); - } - - @Test - public void testAdd() { - // fail("Not yet implemented"); - } - - @Test - public void testFail() { - // fail("Not yet implemented"); - } - - @Test - public void testRemove() { - // fail("Not yet implemented"); - } - -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceTypeTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceTypeTest.java deleted file mode 100644 index b5eb11ffbb4060f2d78f24c74205c1cd0a096018..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceTypeTest.java +++ /dev/null @@ -1,30 +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.faulttolerance; - -import static org.junit.Assert.*; - -import org.junit.Test; - -public class FaultToleranceTypeTest { - - @Test - public void test() { - assertEquals(FaultToleranceType.NONE, FaultToleranceType.from(0)); - assertEquals(FaultToleranceType.AT_LEAST_ONCE, FaultToleranceType.from(1)); - assertEquals(FaultToleranceType.EXACTLY_ONCE, FaultToleranceType.from(2)); - } -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceUtilTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceUtilTest.java deleted file mode 100644 index daa725f4e79387d8375bfb90c0b365ce0945a881..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/faulttolerance/FaultToleranceUtilTest.java +++ /dev/null @@ -1,79 +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.faulttolerance; - -import java.util.LinkedList; -import java.util.List; - -import org.junit.Before; -import org.junit.Test; - -import eu.stratosphere.nephele.io.RecordWriter; -import eu.stratosphere.streaming.api.streamrecord.StreamRecord; - -public class FaultToleranceUtilTest { - - FaultToleranceUtil faultTolerancyBuffer; - List> outputs; - - @Before - public void setFaultTolerancyBuffer() { - outputs = new LinkedList>(); - int[] numOfOutputchannels = { 1, 2 }; - faultTolerancyBuffer = new FaultToleranceUtil(FaultToleranceType.EXACTLY_ONCE, outputs, 1, numOfOutputchannels); - } - - @Test - public void testFaultTolerancyBuffer() { - - } - - @Test - public void testAddRecord() { - - } - - @Test - public void testAddTimestamp() { - - } - - @Test - public void testPopRecord() { - - } - - @Test - public void testRemoveRecord() { - - } - - @Test - public void testAckRecord() { - - } - - @Test - public void testFailRecord() { - - } - - // TODO: create more tests for this method - @Test - public void testTimeOutRecords() { - - } -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/index/BTreeIndexTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/index/BTreeIndexTest.java deleted file mode 100644 index f361af31e7043d64bd09cb47129d32cde6b05ffb..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/index/BTreeIndexTest.java +++ /dev/null @@ -1,35 +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.index; - -import org.junit.Test; - -import eu.stratosphere.streaming.index.BTreeIndex; -import eu.stratosphere.streaming.index.IndexPair; - -public class BTreeIndexTest { - - @Test - public void bTreeIndexOperationTest(){ - BTreeIndex btree=new BTreeIndex(); - btree.put("abc", new IndexPair(7, 3)); - btree.put("abc", new IndexPair(1, 2)); - btree.put("def", new IndexPair(6, 3)); - btree.put("ghi", new IndexPair(3, 6)); - btree.put("jkl", new IndexPair(4, 7)); - System.out.println(btree.get("abc").blockId+", "+btree.get("abc").entryId); - } -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/partitioner/BroadcastPartitionerTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/partitioner/BroadcastPartitionerTest.java deleted file mode 100644 index fdf22a7ef29470cb8b5fc8d7338a5f7ef69e03ec..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/partitioner/BroadcastPartitionerTest.java +++ /dev/null @@ -1,46 +0,0 @@ -/*********************************************************************************************************************** - * - * Copyright (C) 2010-2014 by the Stratosphere project (http://stratosphere.eu) - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - * - **********************************************************************************************************************/ - -package eu.stratosphere.streaming.partitioner; - -import static org.junit.Assert.assertArrayEquals; - -import org.junit.Before; -import org.junit.Test; - -import eu.stratosphere.streaming.api.streamrecord.ArrayStreamRecord; -import eu.stratosphere.streaming.api.streamrecord.StreamRecord; - -public class BroadcastPartitionerTest { - - private BroadcastPartitioner broadcastPartitioner; - private StreamRecord streamRecord = new ArrayStreamRecord(); - - @Before - public void setPartitioner() { - broadcastPartitioner = new BroadcastPartitioner(); - } - - @Test - public void testSelectChannels() { - int[] first = new int[] { 0 }; - int[] second = new int[] { 0, 1 }; - int[] sixth = new int[] { 0, 1, 2, 3, 4, 5 }; - - assertArrayEquals(first, broadcastPartitioner.selectChannels(streamRecord, 1)); - assertArrayEquals(second, broadcastPartitioner.selectChannels(streamRecord, 2)); - assertArrayEquals(sixth, broadcastPartitioner.selectChannels(streamRecord, 6)); - } -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/partitioner/DefaultPartitionerTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/partitioner/DefaultPartitionerTest.java deleted file mode 100644 index 5644b823d2b397211a6ae2e8fbbd192fb4a4e2b4..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/partitioner/DefaultPartitionerTest.java +++ /dev/null @@ -1,58 +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.partitioner; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import org.junit.Before; -import org.junit.Test; - -import eu.stratosphere.streaming.api.streamrecord.ArrayStreamRecord; -import eu.stratosphere.streaming.api.streamrecord.StreamRecord; - -//Currently implemented as a ShufflePartitioner -public class DefaultPartitionerTest { - - private DefaultPartitioner defaultPartitioner; - private StreamRecord streamRecord = new ArrayStreamRecord(); - - @Before - public void setPartitioner() { - defaultPartitioner = new DefaultPartitioner(); - } - - @Test - public void testSelectChannelsLength() { - assertEquals(1, - defaultPartitioner.selectChannels(streamRecord, 1).length); - assertEquals(1, - defaultPartitioner.selectChannels(streamRecord, 2).length); - assertEquals(1, - defaultPartitioner.selectChannels(streamRecord, 1024).length); - } - - @Test - public void testSelectChannelsInterval() { - assertEquals(0, defaultPartitioner.selectChannels(streamRecord, 1)[0]); - - assertTrue(0 <= defaultPartitioner.selectChannels(streamRecord, 2)[0]); - assertTrue(2 > defaultPartitioner.selectChannels(streamRecord, 2)[0]); - - assertTrue(0 <= defaultPartitioner.selectChannels(streamRecord, 1024)[0]); - assertTrue(1024 > defaultPartitioner.selectChannels(streamRecord, 1024)[0]); - } -} \ No newline at end of file diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/partitioner/FieldsPartitionerTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/partitioner/FieldsPartitionerTest.java deleted file mode 100644 index f9b888dc9afaee1fa830244ac69a7a4b0eebfa6c..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/partitioner/FieldsPartitionerTest.java +++ /dev/null @@ -1,59 +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.partitioner; - -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; - -import org.junit.Before; -import org.junit.Test; - -import eu.stratosphere.api.java.tuple.Tuple2; -import eu.stratosphere.streaming.api.streamrecord.ArrayStreamRecord; -import eu.stratosphere.streaming.api.streamrecord.StreamRecord; - -public class FieldsPartitionerTest { - - private FieldsPartitioner fieldsPartitioner; - private StreamRecord streamRecord1 = new ArrayStreamRecord(1).setTuple(0, new Tuple2("test", 0)); - private StreamRecord streamRecord2 = new ArrayStreamRecord(1).setTuple(0, new Tuple2("test", 42)); - - @Before - public void setPartitioner() { - fieldsPartitioner = new FieldsPartitioner(0); - } - - @Test - public void testSelectChannelsLength() { - assertEquals(1, - fieldsPartitioner.selectChannels(streamRecord1, 1).length); - assertEquals(1, - fieldsPartitioner.selectChannels(streamRecord1, 2).length); - assertEquals(1, - fieldsPartitioner.selectChannels(streamRecord1, 1024).length); - } - - @Test - public void testSelectChannelsGrouping() { - assertArrayEquals(fieldsPartitioner.selectChannels(streamRecord1, 1), - fieldsPartitioner.selectChannels(streamRecord2, 1)); - assertArrayEquals(fieldsPartitioner.selectChannels(streamRecord1, 2), - fieldsPartitioner.selectChannels(streamRecord2, 2)); - assertArrayEquals( - fieldsPartitioner.selectChannels(streamRecord1, 1024), - fieldsPartitioner.selectChannels(streamRecord2, 1024)); - } -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/partitioner/GlobalPartitionerTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/partitioner/GlobalPartitionerTest.java deleted file mode 100644 index 8e664689fecd4468c65227f748ad9a21a4579059..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/partitioner/GlobalPartitionerTest.java +++ /dev/null @@ -1,47 +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.partitioner; - -import static org.junit.Assert.assertArrayEquals; - -import org.junit.Before; -import org.junit.Test; - -import eu.stratosphere.streaming.api.streamrecord.ArrayStreamRecord; -import eu.stratosphere.streaming.api.streamrecord.StreamRecord; - -public class GlobalPartitionerTest { - - private GlobalPartitioner globalPartitioner; - private StreamRecord streamRecord = new ArrayStreamRecord(); - - @Before - public void setPartitioner() { - globalPartitioner = new GlobalPartitioner(); - } - - @Test - public void testSelectChannels() { - int[] result = new int[] { 0 }; - - assertArrayEquals(result, - globalPartitioner.selectChannels(streamRecord, 1)); - assertArrayEquals(result, - globalPartitioner.selectChannels(streamRecord, 2)); - assertArrayEquals(result, - globalPartitioner.selectChannels(streamRecord, 1024)); - } -} \ No newline at end of file diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/partitioner/ShufflePartitionerTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/partitioner/ShufflePartitionerTest.java deleted file mode 100644 index cece67db03456b86e8ba4834bc34bcbf2cda8696..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/partitioner/ShufflePartitionerTest.java +++ /dev/null @@ -1,57 +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.partitioner; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import org.junit.Before; -import org.junit.Test; - -import eu.stratosphere.streaming.api.streamrecord.ArrayStreamRecord; -import eu.stratosphere.streaming.api.streamrecord.StreamRecord; - -public class ShufflePartitionerTest { - - private ShufflePartitioner shufflePartitioner; - private StreamRecord streamRecord = new ArrayStreamRecord(); - - @Before - public void setPartitioner() { - shufflePartitioner = new ShufflePartitioner(); - } - - @Test - public void testSelectChannelsLength() { - assertEquals(1, - shufflePartitioner.selectChannels(streamRecord, 1).length); - assertEquals(1, - shufflePartitioner.selectChannels(streamRecord, 2).length); - assertEquals(1, - shufflePartitioner.selectChannels(streamRecord, 1024).length); - } - - @Test - public void testSelectChannelsInterval() { - assertEquals(0, shufflePartitioner.selectChannels(streamRecord, 1)[0]); - - assertTrue(0 <= shufflePartitioner.selectChannels(streamRecord, 2)[0]); - assertTrue(2 > shufflePartitioner.selectChannels(streamRecord, 2)[0]); - - assertTrue(0 <= shufflePartitioner.selectChannels(streamRecord, 1024)[0]); - assertTrue(1024 > shufflePartitioner.selectChannels(streamRecord, 1024)[0]); - } -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/state/InternalStateTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/state/InternalStateTest.java deleted file mode 100644 index e067965d1b861ad1df28c8e78ffa3ef02362117c..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/state/InternalStateTest.java +++ /dev/null @@ -1,60 +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.state; - -import org.junit.Test; - -import eu.stratosphere.api.java.tuple.Tuple2; -import eu.stratosphere.streaming.state.MutableTableState; -import eu.stratosphere.streaming.state.TableStateIterator; -import eu.stratosphere.streaming.state.SlidingWindowState; - -public class InternalStateTest { - - @Test - public void MutableTableStateTest(){ - MutableTableState state=new MutableTableState(); - state.put("abc", "hello"); - state.put("test", "world"); - state.put("state", "mutable"); - state.put("streaming", "persist"); - String s=state.get("streaming"); - if(s==null){ - System.out.println("key does not exist!"); - } - else{ - System.out.println("value="+s); - } - s=state.get("null"); - if(s==null){ - System.out.println("key does not exist!"); - } - else{ - System.out.println("value="+s); - } - TableStateIterator iterator=state.getIterator(); - while(iterator.hasNext()){ - Tuple2 tuple=iterator.next(); - System.out.println(tuple.getField(0)+", "+tuple.getField(1)); - } - } - - @Test - public void WindowStateTest(){ - SlidingWindowState state=new SlidingWindowState(100, 20, 10); - - } -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/util/MockRecordWriterFactory.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/util/MockRecordWriterFactory.java deleted file mode 100644 index cabc62390ff90a78ca79e90b3d712f1fb2350335..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/util/MockRecordWriterFactory.java +++ /dev/null @@ -1,38 +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.util; - -import static org.mockito.Mockito.doCallRealMethod; -import static org.mockito.Mockito.mock; - -import org.mockito.Mockito; - -import eu.stratosphere.streaming.api.streamcomponent.MockRecordWriter; -import eu.stratosphere.streaming.api.streamrecord.StreamRecord; - -public class MockRecordWriterFactory { - - public static MockRecordWriter create() { - MockRecordWriter recWriter = mock(MockRecordWriter.class); - - Mockito.when(recWriter.initList()).thenCallRealMethod(); - doCallRealMethod().when(recWriter).emit(Mockito.any(StreamRecord.class)); - - recWriter.initList(); - - return recWriter; - } -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/util/PerformanceTrackerTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/util/PerformanceTrackerTest.java deleted file mode 100755 index 17e2bc1babca4d03c7b162a14bc39b9b07a45592..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/util/PerformanceTrackerTest.java +++ /dev/null @@ -1,112 +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.util; - -import static org.junit.Assert.*; - -import org.junit.Test; - -public class PerformanceTrackerTest { - - @Test - public void testPerformanceTracker() { - - // fail("Not yet implemented"); - } - - @Test - public void testTrackLong() { - // fail("Not yet implemented"); - } - - @Test - public void testTrack() { - PerformanceTracker pT = new PerformanceTracker("tracker", ""); - pT.track(); - pT.track(3); - pT.track(1); - - assertEquals(3, pT.timeStamps.size()); - assertEquals(3, pT.values.size()); - - assertEquals(Long.valueOf(1), pT.values.get(0)); - assertEquals(Long.valueOf(3), pT.values.get(1)); - assertEquals(Long.valueOf(1), pT.values.get(2)); - - PerformanceTracker pT2 = new PerformanceTracker("tracker", 10, 2, ""); - pT2.track(1); - pT2.track(3); - pT2.track(1); - pT2.track(3); - - assertEquals(2, pT2.timeStamps.size()); - assertEquals(2, pT2.values.size()); - - assertEquals(Long.valueOf(4), pT2.values.get(0)); - assertEquals(Long.valueOf(4), pT2.values.get(1)); - - System.out.println(pT2); - System.out.println("--------------"); - - } - - @Test - public void testCount() { - PerformanceCounter pC = new PerformanceCounter("counter", ""); - pC.count(); - pC.count(10); - pC.count(); - - assertEquals(3, pC.timeStamps.size()); - assertEquals(3, pC.values.size()); - - assertEquals(Long.valueOf(1), pC.values.get(0)); - assertEquals(Long.valueOf(11), pC.values.get(1)); - assertEquals(Long.valueOf(12), pC.values.get(2)); - - System.out.println(pC); - System.out.println("--------------"); - - PerformanceCounter pT2 = new PerformanceCounter("counter", 1000, 10000, ""); - - for (int i = 0; i < 10000000; i++) { - pT2.count("test"); - } - - assertEquals(1000, pT2.timeStamps.size()); - - // pT2.writeCSV("C:/temp/test.csv"); - - } - - @Test - public void testTimer() throws InterruptedException { - PerformanceTimer pT = new PerformanceTimer("timer", true, ""); - - pT.startTimer(); - Thread.sleep(100); - pT.stopTimer(); - System.out.println(pT.values.get(0)); - - assertEquals(1, pT.timeStamps.size()); - assertEquals(1, pT.values.size()); - - assertTrue(pT.values.get(0) < 200); - System.out.println(pT); - - } - -} diff --git a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/util/TestDataUtilTest.java b/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/util/TestDataUtilTest.java deleted file mode 100644 index 0d2d8e03fee08106b44514b2cd9c04c7294c15ed..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-core/src/test/java/eu/stratosphere/streaming/util/TestDataUtilTest.java +++ /dev/null @@ -1,54 +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.util; - -import static org.junit.Assert.assertTrue; - -import java.io.BufferedReader; -import java.io.FileInputStream; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStreamReader; - -import org.junit.Test; - -public class TestDataUtilTest { - -// @Test -// public void testDownload() throws FileNotFoundException, IOException { -// String fileToDownload = "hamlet.txt"; -// String expectedFile = "hamletTestExpectation.txt"; -// -// TestDataUtil.download(fileToDownload); -// -// assertTrue(compareFile(TestDataUtil.testDataDir + expectedFile, TestDataUtil.testDataDir -// + fileToDownload)); -// } - - public boolean compareFile(String file1, String file2) throws FileNotFoundException, - IOException { - - BufferedReader myInput1 = new BufferedReader(new InputStreamReader(new FileInputStream(file1))); - BufferedReader myInput2 = new BufferedReader(new InputStreamReader(new FileInputStream(file2))); - - String line1, line2; - while ((line1 = myInput1.readLine()) != null && (line2 = myInput2.readLine()) != null) { - if (!line1.equals(line2)) - return false; - } - return true; - } -} \ No newline at end of file diff --git a/flink-addons/flink-streaming/stratosphere-streaming-examples/pom.xml b/flink-addons/flink-streaming/stratosphere-streaming-examples/pom.xml deleted file mode 100644 index 25d69b057f6bb6e1edbedfb91c483e1ce77b1bf2..0000000000000000000000000000000000000000 --- a/flink-addons/flink-streaming/stratosphere-streaming-examples/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 - - - - - -